From 4e56e5385f4f98fdd8412204e76ef9c3a7e5a602 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 27 Jan 2015 19:17:23 -0800 Subject: [PATCH 001/149] Initial commit --- runners/flink/.gitignore | 12 +++ runners/flink/LICENSE | 202 +++++++++++++++++++++++++++++++++++++++ runners/flink/README.md | 2 + 3 files changed, 216 insertions(+) create mode 100644 runners/flink/.gitignore create mode 100644 runners/flink/LICENSE create mode 100644 runners/flink/README.md diff --git a/runners/flink/.gitignore b/runners/flink/.gitignore new file mode 100644 index 000000000000..32858aad3c38 --- /dev/null +++ b/runners/flink/.gitignore @@ -0,0 +1,12 @@ +*.class + +# Mobile Tools for Java (J2ME) +.mtj.tmp/ + +# Package Files # +*.jar +*.war +*.ear + +# virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml +hs_err_pid* diff --git a/runners/flink/LICENSE b/runners/flink/LICENSE new file mode 100644 index 000000000000..e06d2081865a --- /dev/null +++ b/runners/flink/LICENSE @@ -0,0 +1,202 @@ +Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + Licensed 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. + diff --git a/runners/flink/README.md b/runners/flink/README.md new file mode 100644 index 000000000000..c6cf8487f985 --- /dev/null +++ b/runners/flink/README.md @@ -0,0 +1,2 @@ +# flink-dataflow +Google Dataflow Runner for Apache Flink From 38c0c973793b1a53d579737e5c8d0a590c21ede5 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 27 Jan 2015 19:20:25 -0800 Subject: [PATCH 002/149] Update .gitignore --- runners/flink/.gitignore | 25 +++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/runners/flink/.gitignore b/runners/flink/.gitignore index 32858aad3c38..f9b86224eb52 100644 --- a/runners/flink/.gitignore +++ b/runners/flink/.gitignore @@ -1,12 +1,17 @@ +.cache +.classpath +.idea +.metadata +.settings +.project +.version.properties +filter.properties +target +tmp *.class - -# Mobile Tools for Java (J2ME) -.mtj.tmp/ - -# Package Files # +*.iml +*.swp *.jar -*.war -*.ear - -# virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml -hs_err_pid* +*.log +.DS_Store +_site From 3dccdb6b02cfae258fddcf6aef047a21a3721d2e Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 11 Feb 2015 14:57:24 +0100 Subject: [PATCH 003/149] Initial stubs --- runners/flink/LICENSE | 2 +- runners/flink/pom.xml | 226 ++++++++++++++++++ .../flink/dataflow/ExecutionRunnerResult.java | 32 +++ .../dataflow/FlinkLocalPipelineRunner.java | 80 +++++++ .../flink/dataflow/FlinkRunnerResult.java | 14 ++ .../example/GoogleWordCountExampleRunner.java | 33 +++ .../FlinkTransformTranslators.java | 78 ++++++ .../dataflow/translation/FlinkTranslator.java | 78 ++++++ .../TransformToFlinkOpTranslator.java | 10 + .../translation/TranslationContext.java | 46 ++++ .../flink/src/main/resources/log4j.properties | 21 ++ 11 files changed, 619 insertions(+), 1 deletion(-) create mode 100644 runners/flink/pom.xml create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/ExecutionRunnerResult.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkLocalPipelineRunner.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TransformToFlinkOpTranslator.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java create mode 100644 runners/flink/src/main/resources/log4j.properties diff --git a/runners/flink/LICENSE b/runners/flink/LICENSE index e06d2081865a..e09346158473 100644 --- a/runners/flink/LICENSE +++ b/runners/flink/LICENSE @@ -1,4 +1,4 @@ -Apache License + Apache License Version 2.0, January 2004 http://www.apache.org/licenses/ diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml new file mode 100644 index 000000000000..b26251c62249 --- /dev/null +++ b/runners/flink/pom.xml @@ -0,0 +1,226 @@ + + + + + 4.0.0 + + com.dataartisans + flink-dataflow + 1.0-SNAPSHOT + + Flink Dataflow Runner + jar + + https://github.com/StephanEwen/flink-dataflow + 2015 + + + + The Apache Software License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + https://github.com/StephanEwen/flink-dataflow + git@github.com:StephanEwen/flink-dataflow.git + git@github.com:StephanEwen/flink-dataflow.git + + + + UTF-8 + UTF-8 + 0.9-SNAPSHOT + + + + + org.apache.flink + flink-core + ${flink.version} + + + org.apache.flink + flink-java + ${flink.version} + + + org.apache.flink + flink-clients + ${flink.version} + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + manual_build + + + org.slf4j + slf4j-jdk14 + + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-examples-all + manual_build + + + org.slf4j + slf4j-jdk14 + + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + true + true + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.7 + 1.7 + + + + + + maven-failsafe-plugin + 2.17 + + + + integration-test + verify + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.17 + + + + + org.apache.maven.plugins + maven-eclipse-plugin + 2.8 + + + org.eclipse.jdt.launching.JRE_CONTAINER + + true + true + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.3.1 + + + enforce-maven + + enforce + + + + + + [3.0.3,) + + + + + + + + + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + [2.12.1,) + + check + + + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + [1.0.0,) + + enforce + + + + + + + + + + + + + + + + diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/ExecutionRunnerResult.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/ExecutionRunnerResult.java new file mode 100644 index 000000000000..d88a2aab9981 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/ExecutionRunnerResult.java @@ -0,0 +1,32 @@ +package com.dataartisans.flink.dataflow; + +import java.util.Collections; +import java.util.Map; + +public class ExecutionRunnerResult implements FlinkRunnerResult { + + private final Map aggregators; + + private final long runtime; + + + public ExecutionRunnerResult(Map aggregators, long runtime) { + this.aggregators = (aggregators == null || aggregators.isEmpty()) ? + Collections.emptyMap() : + Collections.unmodifiableMap(aggregators); + + this.runtime = runtime; + } + + // -------------------------------------------------------------------------------------------- + + @Override + public Map getAggregators() { + return aggregators; + } + + @Override + public long getNetRuntime() { + return runtime; + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkLocalPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkLocalPipelineRunner.java new file mode 100644 index 000000000000..632aac61f608 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkLocalPipelineRunner.java @@ -0,0 +1,80 @@ +package com.dataartisans.flink.dataflow; + +import java.util.Map; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.dataartisans.flink.dataflow.translation.FlinkTranslator; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; + + +public class FlinkLocalPipelineRunner extends PipelineRunner { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkLocalPipelineRunner.class); + + + public FlinkLocalPipelineRunner(PipelineOptions options) {} + + // -------------------------------------------------------------------------------------------- + // Run methods + // -------------------------------------------------------------------------------------------- + + @Override + public FlinkRunnerResult run(Pipeline pipeline) { + return run(pipeline, -1); + } + + public FlinkRunnerResult run(Pipeline pipeline, int parallelism) { + if (parallelism <= 0 && parallelism != -1) { + throw new IllegalArgumentException("Parallelism must be positive or -1 for default"); + } + + LOG.info("Executing pipeline using the FlinkLocalPipelineRunner."); + + ExecutionEnvironment env = parallelism == -1 ? + ExecutionEnvironment.createLocalEnvironment() : + ExecutionEnvironment.createLocalEnvironment(parallelism); + + LOG.info("Translating pipeline to Flink program."); + + FlinkTranslator translator = new FlinkTranslator(env); + translator.translate(pipeline); + + LOG.info("Starting execution of Flink program."); + + JobExecutionResult result; + try { + result = env.execute(); + } + catch (Exception e) { + LOG.error("Pipeline execution failed", e); + throw new RuntimeException("Pipeline execution failed", e); + } + + LOG.info("Execution finished in {} msecs", result.getNetRuntime()); + + Map accumulators = result.getAllAccumulatorResults(); + if (accumulators != null && !accumulators.isEmpty()) { + LOG.info("Final aggregator values:"); + + for (Map.Entry entry : result.getAllAccumulatorResults().entrySet()) { + LOG.info("{} : {}", entry.getKey(), entry.getValue()); + } + } + + return new ExecutionRunnerResult(accumulators, result.getNetRuntime()); + } + + // -------------------------------------------------------------------------------------------- + // Factory + // -------------------------------------------------------------------------------------------- + + public static FlinkLocalPipelineRunner fromOptions(PipelineOptions options) { + return new FlinkLocalPipelineRunner(options); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java new file mode 100644 index 000000000000..0035a0617391 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java @@ -0,0 +1,14 @@ +package com.dataartisans.flink.dataflow; + + +import java.util.Map; + +import com.google.cloud.dataflow.sdk.PipelineResult; + + +public interface FlinkRunnerResult extends PipelineResult { + + Map getAggregators(); + + long getNetRuntime(); +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java new file mode 100644 index 000000000000..036242ca378e --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java @@ -0,0 +1,33 @@ +package com.dataartisans.flink.dataflow.example; + +import com.dataartisans.flink.dataflow.FlinkLocalPipelineRunner; +import com.google.cloud.dataflow.examples.WordCount.CountWords; +import com.google.cloud.dataflow.examples.WordCount.Options; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; + +public class GoogleWordCountExampleRunner { + + public static void main(String[] args) { + + String[] arguments = { + String.format("--output=%s/output.txt", System.getProperty("java.io.tmpdir")) + }; + + Options options = PipelineOptionsFactory.fromArgs(arguments) + .withValidation().as(Options.class); + options.setRunner(FlinkLocalPipelineRunner.class); + + Pipeline p = Pipeline.create(options); + + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(new CountWords()) + .apply(TextIO.Write.named("WriteCounts") + .to(options.getOutput()) + .withNumShards(options.getNumShards())); + + p.run(); + } + +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java new file mode 100644 index 000000000000..ba407a6a47cc --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -0,0 +1,78 @@ +package com.dataartisans.flink.dataflow.translation; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.flink.api.java.operators.DataSource; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.io.TextIO.Read.Bound; +import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; + + +public class FlinkTransformTranslators { + + // -------------------------------------------------------------------------------------------- + // Transform Translator Registry + // -------------------------------------------------------------------------------------------- + + @SuppressWarnings("rawtypes") + private static final Map, TransformToFlinkOpTranslator> TRANSLATORS = + new HashMap, TransformToFlinkOpTranslator>(); + + // register the known translators + static { + TRANSLATORS.put(TextIO.Read.Bound.class, new ReadUTFTextTranslator()); + TRANSLATORS.put(ParDo.Bound.class, null); + } + + + public static TransformToFlinkOpTranslator getTranslator(PTransform transform) { + return TRANSLATORS.get(transform.getClass()); + } + + + // -------------------------------------------------------------------------------------------- + // Individual Transform Translators + // -------------------------------------------------------------------------------------------- + + private static class ReadUTFTextTranslator implements TransformToFlinkOpTranslator> { + + @Override + public void translateNode(TransformTreeNode node, Bound transform, TranslationContext translation) { + String path = transform.getFilepattern(); + String name = transform.getName(); + Coder coder = transform.getDefaultOutputCoder(transform.getOutput()); + + if (coder != null && coder != TextIO.DEFAULT_TEXT_CODER) { + throw new UnsupportedOperationException("Currently only supports UTF-8 inputs."); + } + + DataSource source = translation.getExecutionEnvironment().readTextFile(path); + if (name != null) { + source = source.name(name); + } + + translation.registerDataSet(source, node); + } + } + + private static class ParallelDoTranslator implements TransformToFlinkOpTranslator> { + + @Override + public void translateNode(TransformTreeNode node, ParDo.Bound transform, TranslationContext context) { + + + } + } + + + // -------------------------------------------------------------------------------------------- + // Miscellaneous + // -------------------------------------------------------------------------------------------- + + private FlinkTransformTranslators() {} +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java new file mode 100644 index 000000000000..02bf2b1f2519 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java @@ -0,0 +1,78 @@ +package com.dataartisans.flink.dataflow.translation; + +import org.apache.flink.api.java.ExecutionEnvironment; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor; +import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.values.PValue; + + +public class FlinkTranslator implements PipelineVisitor { + + private final TranslationContext context; + + + public FlinkTranslator(ExecutionEnvironment env) { + this.context = new TranslationContext(env); + } + + + public void translate(Pipeline pipeline) { + pipeline.traverseTopologically(this); + } + + + // -------------------------------------------------------------------------------------------- + // Pipeline Visitor Methods + // -------------------------------------------------------------------------------------------- + + @Override + public void enterCompositeTransform(TransformTreeNode node) { + System.out.println("-enterCompositeTransform- " + node); + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + System.out.println("-leaveCompositeTransform- " + node); + } + + @Override + public void visitTransform(TransformTreeNode node) { + // the transformation applied in this node + PTransform transform = node.getTransform(); + + // the translator to the Flink operation(s) + TransformToFlinkOpTranslator translator = FlinkTransformTranslators.getTranslator(transform); + + if (translator == null) { + throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); + } + + applyTransform(transform, node, translator); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + System.out.println("-visitValue- value=" + value + " producer=" + producer); + } + + /** + * Utility method to define a generic variable to cast the translator and the transform to. + * + * @param transform + * @param node + * @param translator + */ + private > void applyTransform(PTransform transform, TransformTreeNode node, TransformToFlinkOpTranslator translator) { + + @SuppressWarnings("unchecked") + T typedTransform = (T) transform; + + @SuppressWarnings("unchecked") + TransformToFlinkOpTranslator typedTranslator = (TransformToFlinkOpTranslator) translator; + + typedTranslator.translateNode(node, typedTransform, context); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TransformToFlinkOpTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TransformToFlinkOpTranslator.java new file mode 100644 index 000000000000..303fab274397 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TransformToFlinkOpTranslator.java @@ -0,0 +1,10 @@ +package com.dataartisans.flink.dataflow.translation; + +import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.transforms.PTransform; + + +public interface TransformToFlinkOpTranslator> { + + void translateNode(TransformTreeNode node, Type transform, TranslationContext context); +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java new file mode 100644 index 000000000000..fe4697114c89 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -0,0 +1,46 @@ +package com.dataartisans.flink.dataflow.translation; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; + +import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; + +public class TranslationContext { + + private final Map> dataSets; + + private final ExecutionEnvironment env; + + // ------------------------------------------------------------------------ + + public TranslationContext(ExecutionEnvironment env) { + this.env = env; + this.dataSets = new HashMap>(); + } + + // ------------------------------------------------------------------------ + + public ExecutionEnvironment getExecutionEnvironment() { + return env; + } + + public DataSet getDataSet(TransformTreeNode node) { + @SuppressWarnings("unchecked") + DataSet typedSet = (DataSet) dataSets.get(node); + return typedSet; + } + + public void registerDataSet(DataSet dataSet, TransformTreeNode node) { + DataSet previous = dataSets.put(node, dataSet); + + if (previous != null) { + // undo the action + dataSets.put(node, previous); + throw new IllegalArgumentException( + "Context contains already a DataSet as the result of the given TreeTransformNode."); + } + } +} diff --git a/runners/flink/src/main/resources/log4j.properties b/runners/flink/src/main/resources/log4j.properties new file mode 100644 index 000000000000..c88022c19960 --- /dev/null +++ b/runners/flink/src/main/resources/log4j.properties @@ -0,0 +1,21 @@ +################################################################################ +# Copyright 2015 Stephan Ewen, dataArtisans +# +# Licensed 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. +################################################################################ + +log4j.rootLogger=INFO,console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n From 2309ab367b9839d058c20d46ce0a2e0a2d0d7327 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 12 Feb 2015 11:45:25 +0100 Subject: [PATCH 004/149] change to latest dataflow sdk version --- runners/flink/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index b26251c62249..fbf36133dc70 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -69,7 +69,7 @@ Copyright 2015 Stephan Ewen, dataArtisans com.google.cloud.dataflow google-cloud-dataflow-java-sdk-all - manual_build + LATEST org.slf4j @@ -80,7 +80,7 @@ Copyright 2015 Stephan Ewen, dataArtisans com.google.cloud.dataflow google-cloud-dataflow-java-examples-all - manual_build + LATEST org.slf4j From a286e1af114e6832699d8e4d17e5d0faa6dd0ae4 Mon Sep 17 00:00:00 2001 From: Max Date: Sun, 15 Feb 2015 20:26:20 +0100 Subject: [PATCH 005/149] Parallel Do Function --- .../example/GoogleWordCountExampleRunner.java | 10 +- .../translation/FlinkDoFnFunction.java | 112 ++++++++++++++++++ .../FlinkTransformTranslators.java | 32 +++-- .../dataflow/translation/FlinkTranslator.java | 22 +++- 4 files changed, 156 insertions(+), 20 deletions(-) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkDoFnFunction.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java index 036242ca378e..bce09ce6e004 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java @@ -11,14 +11,12 @@ public class GoogleWordCountExampleRunner { public static void main(String[] args) { - String[] arguments = { - String.format("--output=%s/output.txt", System.getProperty("java.io.tmpdir")) - }; + Options options = PipelineOptionsFactory.create().as(Options.class); + options.setOutput("/tmp/output2.txt"); - Options options = PipelineOptionsFactory.fromArgs(arguments) - .withValidation().as(Options.class); + //options.setRunner(DirectPipelineRunner.class); options.setRunner(FlinkLocalPipelineRunner.class); - + Pipeline p = Pipeline.create(options); p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkDoFnFunction.java new file mode 100644 index 000000000000..3e04f87d93f9 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkDoFnFunction.java @@ -0,0 +1,112 @@ +package com.dataartisans.flink.dataflow.translation; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.collect.ImmutableList; +import org.apache.flink.api.common.functions.RichMapPartitionFunction; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.util.Collection; + +/** + * Encapsulates a DoFn inside a Flink MapPartitionFunction + */ +public class FlinkDoFnFunction extends RichMapPartitionFunction { + + DoFn doFn; + + public FlinkDoFnFunction(DoFn doFn) { + this.doFn = doFn; + } + + @Override + public void mapPartition(Iterable values, Collector out) throws Exception { + ProcessContext context = new ProcessContext(doFn); + this.doFn.startBundle(context); + for (IN value : values) { + context.inValue = value; + doFn.processElement(context); + out.collect(context.outValue); + } + this.doFn.finishBundle(context); + } + + private class ProcessContext extends DoFn.ProcessContext { + + IN inValue; + OUT outValue; + + public ProcessContext(DoFn fn) { + fn.super(); + } + + @Override + public IN element() { + return this.inValue; + } + + @Override + public DoFn.KeyedState keyedState() { + throw new UnsupportedOperationException("Getting the keyed state is not supported!"); + } + + @Override + public Instant timestamp() { + return Instant.now(); + } + + @Override + public Collection windows() { + return ImmutableList.of(); + } + + @Override + public PipelineOptions getPipelineOptions() { + throw new UnsupportedOperationException("PipelineOptions are not yet supported!"); + } + + @Override + public T sideInput(PCollectionView view) { + return null; + } + + @Override + public void output(OUT output) { + this.outValue = output; + } + + @Override + public void outputWithTimestamp(OUT output, Instant timestamp) { + // not FLink's way, just output normally + output(output); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + throw new UnsupportedOperationException("Side outputs are not supported!"); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + sideOutput(tag, output); + } + + @Override + public Aggregator createAggregator(String name, Combine.CombineFn combiner) { + //RuntimeContext con = FlinkDoFnFunction.this.getRuntimeContext(); + throw new UnsupportedOperationException("Needs to be implemented!"); + } + + @Override + public Aggregator createAggregator(String name, SerializableFunction, AO> combiner) { + throw new UnsupportedOperationException("Needs to be implemented!"); + } + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index ba407a6a47cc..0f40d4174570 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -1,16 +1,18 @@ package com.dataartisans.flink.dataflow.translation; -import java.util.HashMap; -import java.util.Map; - -import org.apache.flink.api.java.operators.DataSource; - import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.io.TextIO.Read.Bound; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.operators.DataSource; + +import java.util.HashMap; +import java.util.Map; public class FlinkTransformTranslators { @@ -26,7 +28,7 @@ public class FlinkTransformTranslators { // register the known translators static { TRANSLATORS.put(TextIO.Read.Bound.class, new ReadUTFTextTranslator()); - TRANSLATORS.put(ParDo.Bound.class, null); + TRANSLATORS.put(ParDo.Bound.class, new ParallelDoTranslator()); } @@ -42,7 +44,7 @@ public static TransformToFlinkOpTranslator getTranslator(PTransform tra private static class ReadUTFTextTranslator implements TransformToFlinkOpTranslator> { @Override - public void translateNode(TransformTreeNode node, Bound transform, TranslationContext translation) { + public void translateNode(TransformTreeNode node, Bound transform, TranslationContext context) { String path = transform.getFilepattern(); String name = transform.getName(); Coder coder = transform.getDefaultOutputCoder(transform.getOutput()); @@ -51,12 +53,12 @@ public void translateNode(TransformTreeNode node, Bound transform, Trans throw new UnsupportedOperationException("Currently only supports UTF-8 inputs."); } - DataSource source = translation.getExecutionEnvironment().readTextFile(path); + DataSource source = context.getExecutionEnvironment().readTextFile(path); if (name != null) { source = source.name(name); } - - translation.registerDataSet(source, node); + + context.registerDataSet(source, node); } } @@ -64,8 +66,16 @@ private static class ParallelDoTranslator implements TransformToFlinkOp @Override public void translateNode(TransformTreeNode node, ParDo.Bound transform, TranslationContext context) { + + ExecutionEnvironment env = context.getExecutionEnvironment(); + System.out.println("test: " + node.getInput()); + DataSet in = context.getDataSet(node); + System.out.println(in); + final DoFn doFn = transform.getFn(); + + in.mapPartition(new FlinkDoFnFunction<>(doFn)); - + context.registerDataSet(in, node); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java index 02bf2b1f2519..850bbfbd86e3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java @@ -28,18 +28,32 @@ public void translate(Pipeline pipeline) { // Pipeline Visitor Methods // -------------------------------------------------------------------------------------------- + private String genSpaces(int n) { + String s = ""; + for(int i = 0; i < n; i++) { + s += "| "; + } + return s; + } + @Override public void enterCompositeTransform(TransformTreeNode node) { - System.out.println("-enterCompositeTransform- " + node); + System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + node.toString().split("@")[1]); + this.currentCompositeTransformNode = node; + this.depth++; } @Override public void leaveCompositeTransform(TransformTreeNode node) { - System.out.println("-leaveCompositeTransform- " + node); + this.depth--; + System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + node.toString().split("@")[1]); + this.currentCompositeTransformNode = null; } @Override public void visitTransform(TransformTreeNode node) { + System.out.println(genSpaces(this.depth) + "visitTransform- " + node.toString().split("@")[1]); + // the transformation applied in this node PTransform transform = node.getTransform(); @@ -51,11 +65,13 @@ public void visitTransform(TransformTreeNode node) { } applyTransform(transform, node, translator); + } @Override public void visitValue(PValue value, TransformTreeNode producer) { - System.out.println("-visitValue- value=" + value + " producer=" + producer); + // do nothing here + System.out.println(genSpaces(this.depth) + " ^-visitValue- value=" + value + " producer=" + producer); } /** From 92dd104565389e6a39e9c40f4028b10a3c64ee85 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 16 Feb 2015 00:13:46 +0100 Subject: [PATCH 006/149] GroupByKeyOnly --- .../FlinkTransformTranslators.java | 33 ++++++++++--- .../dataflow/translation/FlinkTranslator.java | 20 ++++---- .../translation/TranslationContext.java | 48 +++++++++++-------- 3 files changed, 67 insertions(+), 34 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 0f40d4174570..3d26d7ba6ccc 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -5,10 +5,13 @@ import com.google.cloud.dataflow.sdk.io.TextIO.Read.Bound; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.KV; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.operators.DataSource; import java.util.HashMap; @@ -29,6 +32,7 @@ public class FlinkTransformTranslators { static { TRANSLATORS.put(TextIO.Read.Bound.class, new ReadUTFTextTranslator()); TRANSLATORS.put(ParDo.Bound.class, new ParallelDoTranslator()); + TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslator()); } @@ -45,6 +49,8 @@ private static class ReadUTFTextTranslator implements TransformToFlinkOpTranslat @Override public void translateNode(TransformTreeNode node, Bound transform, TranslationContext context) { + System.out.println("Translating " + node.getFullName()); + String path = transform.getFilepattern(); String name = transform.getName(); Coder coder = transform.getDefaultOutputCoder(transform.getOutput()); @@ -57,8 +63,22 @@ public void translateNode(TransformTreeNode node, Bound transform, Trans if (name != null) { source = source.name(name); } + context.setOutputDataSet(node, source); + } + } + + private static class GroupByKeyOnlyTranslator implements TransformToFlinkOpTranslator> { - context.registerDataSet(source, node); + @Override + public void translateNode(TransformTreeNode node, GroupByKey.GroupByKeyOnly transform, TranslationContext context) { + DataSet> dataSet = (DataSet>) context.getInputDataSet(node); + dataSet.groupBy(new KeySelector, K>() { + @Override + public K getKey(KV kv) throws Exception { + return kv.getKey(); + } + }); + context.setOutputDataSet(node, dataSet); } } @@ -66,16 +86,15 @@ private static class ParallelDoTranslator implements TransformToFlinkOp @Override public void translateNode(TransformTreeNode node, ParDo.Bound transform, TranslationContext context) { + System.out.println("Translating " + node.getFullName()); ExecutionEnvironment env = context.getExecutionEnvironment(); - System.out.println("test: " + node.getInput()); - DataSet in = context.getDataSet(node); - System.out.println(in); + DataSet dataSet = (DataSet) context.getInputDataSet(node); + final DoFn doFn = transform.getFn(); - - in.mapPartition(new FlinkDoFnFunction<>(doFn)); + dataSet.mapPartition(new FlinkDoFnFunction<>(doFn)); - context.registerDataSet(in, node); + context.setOutputDataSet(node, dataSet); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java index 850bbfbd86e3..ff6ac9be419f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java @@ -13,6 +13,8 @@ public class FlinkTranslator implements PipelineVisitor { private final TranslationContext context; + private int depth = 0; + public FlinkTranslator(ExecutionEnvironment env) { this.context = new TranslationContext(env); @@ -28,7 +30,7 @@ public void translate(Pipeline pipeline) { // Pipeline Visitor Methods // -------------------------------------------------------------------------------------------- - private String genSpaces(int n) { + private static String genSpaces(int n) { String s = ""; for(int i = 0; i < n; i++) { s += "| "; @@ -36,24 +38,26 @@ private String genSpaces(int n) { return s; } + private static String formatNodeName(TransformTreeNode node) { + return node.toString().split("@")[1] + node.getFullName(); + } + @Override public void enterCompositeTransform(TransformTreeNode node) { - System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + node.toString().split("@")[1]); - this.currentCompositeTransformNode = node; + System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); this.depth++; } @Override public void leaveCompositeTransform(TransformTreeNode node) { this.depth--; - System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + node.toString().split("@")[1]); - this.currentCompositeTransformNode = null; + System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node)); } @Override public void visitTransform(TransformTreeNode node) { - System.out.println(genSpaces(this.depth) + "visitTransform- " + node.toString().split("@")[1]); - + System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node)); + // the transformation applied in this node PTransform transform = node.getTransform(); @@ -71,7 +75,7 @@ public void visitTransform(TransformTreeNode node) { @Override public void visitValue(PValue value, TransformTreeNode producer) { // do nothing here - System.out.println(genSpaces(this.depth) + " ^-visitValue- value=" + value + " producer=" + producer); + //System.out.println(genSpaces(this.depth) + " ^-visitValue- value=" + value); } /** diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index fe4697114c89..7575b0426f10 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -1,16 +1,16 @@ package com.dataartisans.flink.dataflow.translation; -import java.util.HashMap; -import java.util.Map; - +import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.values.PValue; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import java.util.HashMap; +import java.util.Map; public class TranslationContext { - private final Map> dataSets; + private final Map> dataSets; private final ExecutionEnvironment env; @@ -18,7 +18,7 @@ public class TranslationContext { public TranslationContext(ExecutionEnvironment env) { this.env = env; - this.dataSets = new HashMap>(); + this.dataSets = new HashMap<>(); } // ------------------------------------------------------------------------ @@ -27,20 +27,30 @@ public ExecutionEnvironment getExecutionEnvironment() { return env; } - public DataSet getDataSet(TransformTreeNode node) { - @SuppressWarnings("unchecked") - DataSet typedSet = (DataSet) dataSets.get(node); - return typedSet; +// private DataSet getDataSet(TransformTreeNode node) { +// @SuppressWarnings("unchecked") +// DataSet typedSet = (DataSet) dataSets.get(node); +// return typedSet; +// } + + public DataSet getInputDataSet(TransformTreeNode node) { + PValue value = (PValue) node.getInput(); + return dataSets.get(value); } - public void registerDataSet(DataSet dataSet, TransformTreeNode node) { - DataSet previous = dataSets.put(node, dataSet); - - if (previous != null) { - // undo the action - dataSets.put(node, previous); - throw new IllegalArgumentException( - "Context contains already a DataSet as the result of the given TreeTransformNode."); - } + public void setOutputDataSet(TransformTreeNode node, DataSet value) { + PValue output = (PValue) node.getOutput(); + dataSets.put(output, value); } + +// public void registerDataSet(DataSet dataSet, TransformTreeNode node) { +// DataSet previous = dataSets.put(node, dataSet); +// +// if (previous != null) { +// // undo the action +// dataSets.put(node, previous); +// throw new IllegalArgumentException( +// "Context contains already a DataSet as the result of the given TreeTransformNode."); +// } +// } } From e8e70994755b7794ab49da522fffdf1b2fe6e8f6 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 16 Feb 2015 12:30:39 +0100 Subject: [PATCH 007/149] change flink version to 0.8.0 --- runners/flink/pom.xml | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index fbf36133dc70..5911ad6c02e9 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -26,7 +26,6 @@ Copyright 2015 Stephan Ewen, dataArtisans Flink Dataflow Runner jar - https://github.com/StephanEwen/flink-dataflow 2015 @@ -37,16 +36,10 @@ Copyright 2015 Stephan Ewen, dataArtisans - - https://github.com/StephanEwen/flink-dataflow - git@github.com:StephanEwen/flink-dataflow.git - git@github.com:StephanEwen/flink-dataflow.git - - UTF-8 UTF-8 - 0.9-SNAPSHOT + 0.8.0 From 800025777e9964e327cfd821c802e19e6536bd1d Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 16 Feb 2015 19:01:46 +0100 Subject: [PATCH 008/149] working WordCount and TFIDF --- .../dataflow/FlinkLocalPipelineRunner.java | 13 +- .../flink/dataflow/example/AnotherJOb.java | 39 +++++ .../dataflow/example/FlattenizeThis.java | 104 ++++++++++++ .../example/GoogleWordCountExampleRunner.java | 4 +- .../dataflow/example/MaybeEmptyTest.java | 50 ++++++ .../flink/dataflow/example/SideInputTest.java | 44 +++++ .../flink/dataflow/example/TFIDF.java | 158 ++++++++++++++++++ .../translation/CoderTypeInformation.java | 115 +++++++++++++ .../translation/CoderTypeSerializer.java | 112 +++++++++++++ .../translation/CoderVoidTypeSerializer.java | 71 ++++++++ .../translation/FlinkCreateFlatMap.java | 25 +++ .../translation/FlinkDoFnFunction.java | 39 +++-- .../FlinkTransformTranslators.java | 138 +++++++++++---- .../dataflow/translation/FlinkTranslator.java | 8 +- .../translation/KeyedListAggregator.java | 23 +++ .../translation/KvCoderComperator.java | 145 ++++++++++++++++ ...anslator.java => TransformTranslator.java} | 2 +- .../translation/TranslationContext.java | 56 ++++--- .../utils/CombineFnAggregatorWrapper.java | 74 ++++++++ .../utils/DataInputStreamWrapper.java | 29 ++++ .../utils/DataOutputStreamWrapper.java | 29 ++++ .../SerializableFnAggregatorWrapper.java | 76 +++++++++ 22 files changed, 1272 insertions(+), 82 deletions(-) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/AnotherJOb.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/FlattenizeThis.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/MaybeEmptyTest.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/SideInputTest.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/TFIDF.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderTypeInformation.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderTypeSerializer.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderVoidTypeSerializer.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkCreateFlatMap.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/KeyedListAggregator.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/KvCoderComperator.java rename runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/{TransformToFlinkOpTranslator.java => TransformTranslator.java} (77%) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/CombineFnAggregatorWrapper.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/DataInputStreamWrapper.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/DataOutputStreamWrapper.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/SerializableFnAggregatorWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkLocalPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkLocalPipelineRunner.java index 632aac61f608..d5383d05e743 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkLocalPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkLocalPipelineRunner.java @@ -1,16 +1,15 @@ package com.dataartisans.flink.dataflow; -import java.util.Map; - +import com.dataartisans.flink.dataflow.translation.FlinkTranslator; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.ExecutionEnvironment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.dataartisans.flink.dataflow.translation.FlinkTranslator; -import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import java.util.Map; public class FlinkLocalPipelineRunner extends PipelineRunner { @@ -49,6 +48,8 @@ public FlinkRunnerResult run(Pipeline pipeline, int parallelism) { JobExecutionResult result; try { + env.setDegreeOfParallelism(1); + System.out.println(env.getExecutionPlan()); result = env.execute(); } catch (Exception e) { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/AnotherJOb.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/AnotherJOb.java new file mode 100644 index 000000000000..a2aeb3c162f7 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/AnotherJOb.java @@ -0,0 +1,39 @@ +package com.dataartisans.flink.dataflow.example; + +import com.dataartisans.flink.dataflow.FlinkLocalPipelineRunner; +import com.google.cloud.dataflow.examples.WordCount; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; + +/** + * Created by max on 16/02/15. + */ +public class AnotherJOb { + public static void main(String[] args) { + + WordCount.Options options = PipelineOptionsFactory.create().as(WordCount.Options.class); + options.setOutput("/tmp/output2.txt"); + options.setInput("/Users/max/hello_world.txt"); + //options.setRunner(DirectPipelineRunner.class); + options.setRunner(FlinkLocalPipelineRunner.class); + + Pipeline p = Pipeline.create(options); + + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(ParDo.of(new DoFn() { + + @Override + public void processElement(ProcessContext c) throws Exception { + c.output(c.element()); + } + })) + .apply(TextIO.Write.named("WriteCounts") + .to(options.getOutput()) + .withNumShards(options.getNumShards())); + + p.run(); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/FlattenizeThis.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/FlattenizeThis.java new file mode 100644 index 000000000000..704da25a36e8 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/FlattenizeThis.java @@ -0,0 +1,104 @@ +package com.dataartisans.flink.dataflow.example; + +import com.dataartisans.flink.dataflow.FlinkLocalPipelineRunner; +import com.google.cloud.dataflow.examples.TfIdf; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.*; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.util.GcsUtil; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.cloud.dataflow.sdk.values.KV; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class FlattenizeThis { + + private static interface Options extends PipelineOptions { + @Description("Path to the directory or GCS prefix containing files to read from") + @Default.String("gs://dataflow-samples/shakespeare/") + String getInput(); + void setInput(String value); + + @Description("Prefix of output URI to write to") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + public static Set listInputDocuments(Options options) + throws URISyntaxException, IOException { + URI baseUri = new URI(options.getInput()); + + // List all documents in the directory or GCS prefix. + URI absoluteUri; + if (baseUri.getScheme() != null) { + absoluteUri = baseUri; + } else { + absoluteUri = new URI( + "file", + baseUri.getAuthority(), + baseUri.getPath(), + baseUri.getQuery(), + baseUri.getFragment()); + } + + Set uris = new HashSet<>(); + if (absoluteUri.getScheme().equals("file")) { + File directory = new File(absoluteUri); + for (String entry : directory.list()) { + File path = new File(directory, entry); + uris.add(path.toURI()); + } + } else if (absoluteUri.getScheme().equals("gs")) { + GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil(); + URI gcsUriGlob = new URI( + absoluteUri.getScheme(), + absoluteUri.getAuthority(), + absoluteUri.getPath() + "*", + absoluteUri.getQuery(), + absoluteUri.getFragment()); + for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) { + uris.add(entry.toUri()); + } + } + + return uris; + } + + public static void main(String[] args) throws IOException, URISyntaxException { + Options options = PipelineOptionsFactory.create().as(Options.class); + options.setOutput("/tmp/output2.txt"); + options.setInput("/tmp/documents"); + //options.setRunner(DirectPipelineRunner.class); + options.setRunner(FlinkLocalPipelineRunner.class); + + Pipeline p = Pipeline.create(options); + + p.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); + + List documentURLs = new ArrayList(); + + + p.apply(new TfIdf.ReadDocuments(listInputDocuments(options))) + .apply(ParDo.of(new DoFn, String>() { + @Override + public void processElement(ProcessContext c) throws Exception { + c.output(c.element().toString()); + } + })) + .apply(TextIO.Write.named("WriteCounts") + .to(options.getOutput())); + + p.run(); + } +} \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java index bce09ce6e004..d46924b5e4f5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/GoogleWordCountExampleRunner.java @@ -13,10 +13,10 @@ public static void main(String[] args) { Options options = PipelineOptionsFactory.create().as(Options.class); options.setOutput("/tmp/output2.txt"); - + options.setInput("/tmp/documents/hello_world.txt"); //options.setRunner(DirectPipelineRunner.class); options.setRunner(FlinkLocalPipelineRunner.class); - + Pipeline p = Pipeline.create(options); p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/MaybeEmptyTest.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/MaybeEmptyTest.java new file mode 100644 index 000000000000..00cec4670466 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/MaybeEmptyTest.java @@ -0,0 +1,50 @@ +package com.dataartisans.flink.dataflow.example; + +import com.dataartisans.flink.dataflow.FlinkLocalPipelineRunner; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.*; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; + +/** + * Created by max on 18/02/15. + */ +public class MaybeEmptyTest { + + private static interface Options extends PipelineOptions { + @Description("Path to the directory or GCS prefix containing files to read from") + @Default.String("gs://dataflow-samples/shakespeare/") + String getInput(); + void setInput(String value); + + @Description("Prefix of output URI to write to") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + public static void main(String[] args ){ + Options options = PipelineOptionsFactory.create().as(Options.class); + options.setOutput("/tmp/output2.txt"); + options.setInput("/tmp/documents"); + //options.setRunner(DirectPipelineRunner.class); + options.setRunner(FlinkLocalPipelineRunner.class); + + Pipeline p = Pipeline.create(options); + + p.apply(Create.of((Void) null)).setCoder(VoidCoder.of()) + .apply(ParDo.of( + new DoFn() { + @Override + public void processElement(DoFn.ProcessContext c) { + System.out.println("hello"); + c.output("test"); + } + })).apply(TextIO.Write.to("bla")); + p.run(); + + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/SideInputTest.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/SideInputTest.java new file mode 100644 index 000000000000..d8a336eb5651 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/SideInputTest.java @@ -0,0 +1,44 @@ +package com.dataartisans.flink.dataflow.example; + +import com.dataartisans.flink.dataflow.FlinkLocalPipelineRunner; +import com.google.cloud.dataflow.examples.WordCount; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.values.PCollectionView; + +import java.io.IOException; +import java.net.URISyntaxException; + +public class SideInputTest { + public static void main(String[] args) throws IOException, URISyntaxException { + WordCount.Options options = PipelineOptionsFactory.create().as(WordCount.Options.class); + options.setOutput("/tmp/output2.txt"); + options.setInput("/tmp/documents/hello_world.txt"); + //options.setRunner(DirectPipelineRunner.class); + options.setRunner(FlinkLocalPipelineRunner.class); + + Pipeline p = Pipeline.create(options); + + final PCollectionView totalDocuments = p + .apply(Create.of("Hello!")) + .apply(View.asSingleton()); + + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(ParDo.of(new DoFn() { + + @Override + public void processElement(ProcessContext c) throws Exception { + String s = c.sideInput(totalDocuments); + System.out.println("side Input:" + s); + c.output(c.element()); + } + }).withSideInputs(totalDocuments)).apply(TextIO.Write.to("/tmp/output")); + + p.run(); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/TFIDF.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/TFIDF.java new file mode 100644 index 000000000000..b5f480d8f795 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/example/TFIDF.java @@ -0,0 +1,158 @@ +package com.dataartisans.flink.dataflow.example; + +import com.google.cloud.dataflow.examples.TfIdf; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.*; +import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.util.GcsUtil; +import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +import com.google.common.collect.Iterables; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class TFIDF { + + private static interface Options extends PipelineOptions { + @Description("Path to the directory or GCS prefix containing files to read from") + @Default.String("gs://dataflow-samples/shakespeare/") + String getInput(); + void setInput(String value); + + @Description("Prefix of output URI to write to") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + public static Set listInputDocuments(Options options) + throws URISyntaxException, IOException { + URI baseUri = new URI(options.getInput()); + + // List all documents in the directory or GCS prefix. + URI absoluteUri; + if (baseUri.getScheme() != null) { + absoluteUri = baseUri; + } else { + absoluteUri = new URI( + "file", + baseUri.getAuthority(), + baseUri.getPath(), + baseUri.getQuery(), + baseUri.getFragment()); + } + + Set uris = new HashSet<>(); + if (absoluteUri.getScheme().equals("file")) { + File directory = new File(absoluteUri); + for (String entry : directory.list()) { + File path = new File(directory, entry); + uris.add(path.toURI()); + } + } else if (absoluteUri.getScheme().equals("gs")) { + GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil(); + URI gcsUriGlob = new URI( + absoluteUri.getScheme(), + absoluteUri.getAuthority(), + absoluteUri.getPath() + "*", + absoluteUri.getQuery(), + absoluteUri.getFragment()); + for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) { + uris.add(entry.toUri()); + } + } + + return uris; + } + + static class SimpleCombineFn extends Combine.CombineFn, V> { + /** + * Returns a {@code CombineFn} that uses the given + * {@code SerializableFunction} to combine values. + */ + public static SimpleCombineFn of( + SerializableFunction, V> combiner) { + return new SimpleCombineFn<>(combiner); + } + + /** + * The number of values to accumulate before invoking the combiner + * function to combine them. + */ + private static final int BUFFER_SIZE = 20; + + /** The combiner function. */ + private final SerializableFunction, V> combiner; + + private SimpleCombineFn(SerializableFunction, V> combiner) { + this.combiner = combiner; + } + + @Override + public List createAccumulator() { + return new ArrayList<>(); + } + + @Override + public void addInput(List accumulator, V input) { + accumulator.add(input); + if (accumulator.size() > BUFFER_SIZE) { + V combined = combiner.apply(accumulator); + accumulator.clear(); + accumulator.add(combined); + } + } + + @Override + public List mergeAccumulators(Iterable> accumulators) { + List singleton = new ArrayList<>(); + singleton.add(combiner.apply(Iterables.concat(accumulators))); + return singleton; + } + + @Override + public V extractOutput(List accumulator) { + return combiner.apply(accumulator); + } + } + + public static void main(String[] args) throws IOException, URISyntaxException { + Options options = PipelineOptionsFactory.create().as(Options.class); + options.setOutput("/tmp/output2.txt"); + options.setInput("/tmp/documents"); + options.setRunner(DirectPipelineRunner.class); + //options.setRunner(FlinkLocalPipelineRunner.class); + + Pipeline p = Pipeline.create(options); + + p.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); + + List documentURLs = new ArrayList(); + + + p.apply(new TfIdf.ReadDocuments(listInputDocuments(options))) + .apply(new TfIdf.ComputeTfIdf()) + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + c.output(c.element().toString()); + } + })) + .apply(TextIO.Write.to("/tmp/output")); + //.apply(new TfIdf.WriteTfIdf(options.getOutput())); + + p.run(); + } +} \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderTypeInformation.java new file mode 100644 index 000000000000..b30159d210cc --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderTypeInformation.java @@ -0,0 +1,115 @@ +package com.dataartisans.flink.dataflow.translation; + +import com.google.api.client.repackaged.com.google.common.base.Preconditions; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.CompositeType; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +import java.util.List; + +public class CoderTypeInformation extends CompositeType { + + private Coder coder; + + public CoderTypeInformation(Coder coder) { + super(null); + this.coder = coder; + Preconditions.checkNotNull(coder); + } + + public TypeComparator createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset) { + return new KvCoderComperator((KvCoder) coder); + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 0; + } + + @Override + public Class getTypeClass() { + return (Class) Object.class; + } + + @Override + public boolean isKeyType() { + return true; + } + + @Override + public TypeSerializer createSerializer() { + if (coder instanceof VoidCoder) { + return (TypeSerializer) new CoderVoidTypeSerializer(); + } + return new CoderTypeSerializer<>(coder); + } + + @Override + public int getTotalFields() { + return 0; + } + + @Override + public void getKey(String fieldExpression, int offset, List result) { + result.add(new FlatFieldDescriptor(0, BasicTypeInfo.INT_TYPE_INFO)); + } + + @Override + public TypeInformation getTypeAt(int pos) { + return null; + } + + @Override + protected void initializeNewComparator(int localKeyCount) { + + } + + @Override + protected TypeComparator getNewComparator() { + return null; + } + + @Override + protected void addCompareField(int fieldId, TypeComparator comparator) { + + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + CoderTypeInformation that = (CoderTypeInformation) o; + + if (!coder.equals(that.coder)) return false; + + return true; + } + + @Override + public int hashCode() { + return coder.hashCode(); + } + + @Override + public String toString() { + return "CoderTypeInformation{" + + "coder=" + coder + + '}'; + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderTypeSerializer.java new file mode 100644 index 000000000000..fef48bb42947 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderTypeSerializer.java @@ -0,0 +1,112 @@ +package com.dataartisans.flink.dataflow.translation; + +import com.dataartisans.flink.dataflow.translation.utils.DataInputStreamWrapper; +import com.dataartisans.flink.dataflow.translation.utils.DataOutputStreamWrapper; +import com.google.cloud.dataflow.sdk.coders.Coder; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; + + +public class CoderTypeSerializer extends TypeSerializer { + + private Coder coder; + private transient DataInputStreamWrapper inputWrapper; + private transient DataOutputStreamWrapper outputWrapper; + + public CoderTypeSerializer(Coder coder) { + this.coder = coder; + this.inputWrapper = new DataInputStreamWrapper(null); + this.outputWrapper = new DataOutputStreamWrapper(null); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + this.inputWrapper = new DataInputStreamWrapper(null); + this.outputWrapper = new DataOutputStreamWrapper(null); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public boolean isStateful() { + return false; + } + + @Override + public T createInstance() { + return null; + } + + @Override + public T copy(T t) { + ByteArrayOutputStream bao = new ByteArrayOutputStream(); + try { + coder.encode(t, bao, Coder.Context.OUTER); + } catch (IOException e) { + throw new RuntimeException("Could not copy.", e); + } + try { + return coder.decode(new ByteArrayInputStream(bao.toByteArray()), Coder.Context.OUTER); + } catch (IOException e) { + throw new RuntimeException("Could not copy.", e); + } + } + + @Override + public T copy(T t, T reuse) { + return copy(t); + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(T t, DataOutputView dataOutputView) throws IOException { + outputWrapper.setOutputView(dataOutputView); + coder.encode(t, outputWrapper, Coder.Context.NESTED); + } + + @Override + public T deserialize(DataInputView dataInputView) throws IOException { + inputWrapper.setInputView(dataInputView); + return coder.decode(inputWrapper, Coder.Context.NESTED); + } + + @Override + public T deserialize(T t, DataInputView dataInputView) throws IOException { + return deserialize(dataInputView); + } + + @Override + public void copy(DataInputView dataInputView, DataOutputView dataOutputView) throws IOException { + serialize(deserialize(dataInputView), dataOutputView); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + CoderTypeSerializer that = (CoderTypeSerializer) o; + + if (!coder.equals(that.coder)) return false; + + return true; + } + + @Override + public int hashCode() { + return coder.hashCode(); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderVoidTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderVoidTypeSerializer.java new file mode 100644 index 000000000000..3dd7c7e4a0da --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/CoderVoidTypeSerializer.java @@ -0,0 +1,71 @@ +package com.dataartisans.flink.dataflow.translation; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + + +public class CoderVoidTypeSerializer extends TypeSerializer { + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public boolean isStateful() { + return false; + } + + @Override + public VoidValue createInstance() { + return VoidValue.INSTANCE; + } + + @Override + public VoidValue copy(VoidValue from) { + return from; + } + + @Override + public VoidValue copy(VoidValue from, VoidValue reuse) { + return from; + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(VoidValue record, DataOutputView target) throws IOException { + target.writeByte(1); + } + + @Override + public VoidValue deserialize(DataInputView source) throws IOException { + source.readByte(); + return VoidValue.INSTANCE; + } + + @Override + public VoidValue deserialize(VoidValue reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + source.readByte(); + target.writeByte(1); + } + + public static class VoidValue { + private VoidValue() {} + + public static VoidValue INSTANCE = new VoidValue(); + } + + +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkCreateFlatMap.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkCreateFlatMap.java new file mode 100644 index 000000000000..d61c97ad1d2c --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkCreateFlatMap.java @@ -0,0 +1,25 @@ +package com.dataartisans.flink.dataflow.translation; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.util.Collector; + +import java.util.List; + +/** + * Encapsulates a DoFn inside a Flink MapPartitionFunction + */ +public class FlinkCreateFlatMap implements FlatMapFunction { + + private List actualElements; + + public FlinkCreateFlatMap(List actualElements) { + this.actualElements = actualElements; + } + + @Override + public void flatMap(IN value, Collector out) throws Exception { + for (OUT actualValue : actualElements) { + out.collect(actualValue); + } + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkDoFnFunction.java index 3e04f87d93f9..4f201ba12af7 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkDoFnFunction.java @@ -1,19 +1,26 @@ package com.dataartisans.flink.dataflow.translation; +import com.dataartisans.flink.dataflow.translation.utils.CombineFnAggregatorWrapper; +import com.dataartisans.flink.dataflow.translation.utils.SerializableFnAggregatorWrapper; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableList; import org.apache.flink.api.common.functions.RichMapPartitionFunction; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.util.Collector; import org.joda.time.Instant; +import java.util.ArrayList; import java.util.Collection; +import java.util.List; /** * Encapsulates a DoFn inside a Flink MapPartitionFunction @@ -21,19 +28,18 @@ public class FlinkDoFnFunction extends RichMapPartitionFunction { DoFn doFn; - + public FlinkDoFnFunction(DoFn doFn) { this.doFn = doFn; } @Override public void mapPartition(Iterable values, Collector out) throws Exception { - ProcessContext context = new ProcessContext(doFn); + ProcessContext context = new ProcessContext(doFn, out, getRuntimeContext()); this.doFn.startBundle(context); for (IN value : values) { context.inValue = value; doFn.processElement(context); - out.collect(context.outValue); } this.doFn.finishBundle(context); } @@ -41,10 +47,13 @@ public void mapPartition(Iterable values, Collector out) throws Excepti private class ProcessContext extends DoFn.ProcessContext { IN inValue; - OUT outValue; + Collector outCollector; + RuntimeContext runContext; - public ProcessContext(DoFn fn) { + public ProcessContext(DoFn fn, Collector outCollector, RuntimeContext runContext) { fn.super(); + this.outCollector = outCollector; + this.runContext = runContext; } @Override @@ -74,12 +83,17 @@ public PipelineOptions getPipelineOptions() { @Override public T sideInput(PCollectionView view) { - return null; + List sideInput = runContext.getBroadcastVariable(view.getTagInternal().getId()); + List> windowedValueList = new ArrayList<>(sideInput.size()); + for (T input : sideInput) { + windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE))); + } + return view.fromIterableInternal(windowedValueList); } @Override public void output(OUT output) { - this.outValue = output; + outCollector.collect(output); } @Override @@ -100,13 +114,16 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times @Override public Aggregator createAggregator(String name, Combine.CombineFn combiner) { - //RuntimeContext con = FlinkDoFnFunction.this.getRuntimeContext(); - throw new UnsupportedOperationException("Needs to be implemented!"); + CombineFnAggregatorWrapper wrapper = new CombineFnAggregatorWrapper(combiner); + getRuntimeContext().addAccumulator(name, wrapper); + return wrapper; } @Override - public Aggregator createAggregator(String name, SerializableFunction, AO> combiner) { - throw new UnsupportedOperationException("Needs to be implemented!"); + public Aggregator createAggregator(String name, SerializableFunction, AO> serializableFunction) { + SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper(serializableFunction); + getRuntimeContext().addAccumulator(name, wrapper); + return wrapper; } } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 3d26d7ba6ccc..924d206cbf19 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -4,17 +4,19 @@ import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.io.TextIO.Read.Bound; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; -import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.GroupByKey; -import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.*; import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.common.collect.Lists; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.DataSource; +import org.apache.flink.api.java.operators.*; +import org.apache.flink.api.java.operators.Keys; import java.util.HashMap; +import java.util.List; import java.util.Map; @@ -25,18 +27,21 @@ public class FlinkTransformTranslators { // -------------------------------------------------------------------------------------------- @SuppressWarnings("rawtypes") - private static final Map, TransformToFlinkOpTranslator> TRANSLATORS = - new HashMap, TransformToFlinkOpTranslator>(); + private static final Map, TransformTranslator> TRANSLATORS = new HashMap<>(); // register the known translators static { TRANSLATORS.put(TextIO.Read.Bound.class, new ReadUTFTextTranslator()); + TRANSLATORS.put(TextIO.Write.Bound.class, new WriteUTFTextTranslator()); TRANSLATORS.put(ParDo.Bound.class, new ParallelDoTranslator()); TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslator()); + TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); + TRANSLATORS.put(View.CreatePCollectionView.class, new PCollectionViewTranslator()); + TRANSLATORS.put(Create.class, new CreateTranslator()); } - public static TransformToFlinkOpTranslator getTranslator(PTransform transform) { + public static TransformTranslator getTranslator(PTransform transform) { return TRANSLATORS.get(transform.getClass()); } @@ -45,7 +50,7 @@ public static TransformToFlinkOpTranslator getTranslator(PTransform tra // Individual Transform Translators // -------------------------------------------------------------------------------------------- - private static class ReadUTFTextTranslator implements TransformToFlinkOpTranslator> { + private static class ReadUTFTextTranslator implements TransformTranslator> { @Override public void translateNode(TransformTreeNode node, Bound transform, TranslationContext context) { @@ -53,53 +58,116 @@ public void translateNode(TransformTreeNode node, Bound transform, Trans String path = transform.getFilepattern(); String name = transform.getName(); - Coder coder = transform.getDefaultOutputCoder(transform.getOutput()); - + Coder coder = transform.getOutput().getCoder(); + if (coder != null && coder != TextIO.DEFAULT_TEXT_CODER) { throw new UnsupportedOperationException("Currently only supports UTF-8 inputs."); } DataSource source = context.getExecutionEnvironment().readTextFile(path); if (name != null) { - source = source.name(name); + source.name(name); } - context.setOutputDataSet(node, source); + context.setOutputDataSet(transform.getOutput(), source); + } + } + + private static class WriteUTFTextTranslator implements TransformTranslator> { + + @Override + public void translateNode(TransformTreeNode node, TextIO.Write.Bound transform, TranslationContext context) { + DataSet dataSet = context.getInputDataSet(transform.getInput()); + dataSet.print(); } } - private static class GroupByKeyOnlyTranslator implements TransformToFlinkOpTranslator> { + private static class GroupByKeyOnlyTranslator implements TransformTranslator> { @Override - public void translateNode(TransformTreeNode node, GroupByKey.GroupByKeyOnly transform, TranslationContext context) { - DataSet> dataSet = (DataSet>) context.getInputDataSet(node); - dataSet.groupBy(new KeySelector, K>() { - @Override - public K getKey(KV kv) throws Exception { - return kv.getKey(); - } - }); - context.setOutputDataSet(node, dataSet); + public void translateNode(TransformTreeNode node, GroupByKey.GroupByKeyOnly transform, TranslationContext context) { + DataSet> dataSet = context.getInputDataSet(transform.getInput()); + GroupReduceFunction, KV>> groupReduceFunction = new KeyedListAggregator<>(); + + TypeInformation>> typeInformation = context.getTypeInfo(transform.getOutput()); + + Grouping> grouping = new UnsortedGrouping<>(dataSet, new Keys.ExpressionKeys<>(new String[]{""}, dataSet.getType())); + + GroupReduceOperator, KV>> dataSetNew = + new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); + context.setOutputDataSet(transform.getOutput(), dataSetNew); } } - private static class ParallelDoTranslator implements TransformToFlinkOpTranslator> { + private static class ParallelDoTranslator implements TransformTranslator> { @Override public void translateNode(TransformTreeNode node, ParDo.Bound transform, TranslationContext context) { - System.out.println("Translating " + node.getFullName()); + DataSet dataSet = context.getInputDataSet(transform.getInput()); - ExecutionEnvironment env = context.getExecutionEnvironment(); - DataSet dataSet = (DataSet) context.getInputDataSet(node); - final DoFn doFn = transform.getFn(); - dataSet.mapPartition(new FlinkDoFnFunction<>(doFn)); - context.setOutputDataSet(node, dataSet); + TypeInformation typeInformation = context.getTypeInfo(transform.getOutput()); + + FlinkDoFnFunction mapPartitionFunction = new FlinkDoFnFunction<>(doFn); + MapPartitionOperator dataSetNew = new MapPartitionOperator<>(dataSet, typeInformation, mapPartitionFunction, transform.getName()); + + List> sideInputs = transform.getSideInputs(); + // get corresponding Flink broadcast data sets + for(PCollectionView input : sideInputs) { + DataSet broadcastSet = context.getSideInputDataSet(input); + dataSetNew.withBroadcastSet(broadcastSet, input.getTagInternal().getId()); + } + + context.setOutputDataSet(transform.getOutput(), dataSetNew); } } - - - // -------------------------------------------------------------------------------------------- + + private static class FlattenPCollectionTranslator implements TransformTranslator> { + + @Override + public void translateNode(TransformTreeNode node, Flatten.FlattenPCollectionList transform, TranslationContext context) { + List> allInputs = transform.getInput().getAll(); + DataSet result = null; + for(PCollection collection : allInputs) { + DataSet current = context.getInputDataSet(collection); + if (result == null) { + result = current; + } else { + result = result.union(current); + } + } + context.setOutputDataSet(transform.getOutput(), result); + } + } + + private static class PCollectionViewTranslator implements TransformTranslator> { + @Override + public void translateNode(TransformTreeNode node, View.CreatePCollectionView transform, TranslationContext context) { + DataSet dataSet = context.getInputDataSet(transform.getInput()); + PCollectionView input = transform.apply(null); + context.setSideInputDataSet(input, dataSet); + } + } + + private static class CreateTranslator implements TransformTranslator> { + + @Override + public void translateNode(TransformTreeNode node, Create transform, TranslationContext context) { + TypeInformation typeInformation = context.getTypeInfo(transform.getOutput()); + Iterable elements = transform.getElements(); + for(OUT elem : elements) { + System.out.println("element:"+elem); + System.out.println("type: " + typeInformation); + } + DataSet initDataSet = context.getExecutionEnvironment().fromElements(1); + FlinkCreateFlatMap flatMapFunction = new FlinkCreateFlatMap<>(Lists.newArrayList(elements)); + FlatMapOperator dataSetNew = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName()); + + context.setOutputDataSet(transform.getOutput(), dataSetNew); + } + } + + // -------------------------------------------------------------------------------------------- // Miscellaneous // -------------------------------------------------------------------------------------------- diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java index ff6ac9be419f..6579072193b1 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTranslator.java @@ -57,14 +57,14 @@ public void leaveCompositeTransform(TransformTreeNode node) { @Override public void visitTransform(TransformTreeNode node) { System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node)); - // the transformation applied in this node PTransform transform = node.getTransform(); // the translator to the Flink operation(s) - TransformToFlinkOpTranslator translator = FlinkTransformTranslators.getTranslator(transform); + TransformTranslator translator = FlinkTransformTranslators.getTranslator(transform); if (translator == null) { + System.out.println(node.getTransform().getClass()); throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); } @@ -85,13 +85,13 @@ public void visitValue(PValue value, TransformTreeNode producer) { * @param node * @param translator */ - private > void applyTransform(PTransform transform, TransformTreeNode node, TransformToFlinkOpTranslator translator) { + private > void applyTransform(PTransform transform, TransformTreeNode node, TransformTranslator translator) { @SuppressWarnings("unchecked") T typedTransform = (T) transform; @SuppressWarnings("unchecked") - TransformToFlinkOpTranslator typedTranslator = (TransformToFlinkOpTranslator) translator; + TransformTranslator typedTranslator = (TransformTranslator) translator; typedTranslator.translateNode(node, typedTransform, context); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/KeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/KeyedListAggregator.java new file mode 100644 index 000000000000..17c17a0cd24e --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/KeyedListAggregator.java @@ -0,0 +1,23 @@ +package com.dataartisans.flink.dataflow.translation; + +import com.google.cloud.dataflow.sdk.values.KV; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.util.Collector; + +import java.util.ArrayList; +import java.util.List; + +public class KeyedListAggregator implements GroupReduceFunction, KV>> { + + @Override + public void reduce(Iterable> values, Collector>> out) throws Exception { + K k = null; + List result = new ArrayList(); + for (KV kv : values) { + k = kv.getKey(); + result.add(kv.getValue()); + } + out.collect(KV.of(k, (Iterable) result)); + } + +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/KvCoderComperator.java new file mode 100644 index 000000000000..87855334a11f --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/KvCoderComperator.java @@ -0,0 +1,145 @@ +package com.dataartisans.flink.dataflow.translation; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.values.KV; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.MemorySegment; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +public class KvCoderComperator extends TypeComparator> { + + private KV reference = null; + private KvCoder coder; + + public KvCoderComperator(KvCoder coder) { + this.coder = coder; + } + + public KV getReference() { + return reference; + } + + @Override + public int hash(KV record) { + K key = record.getKey(); + if (key != null) { + return key.hashCode(); + } else { + return 0; + } + } + + @Override + public void setReference(KV toCompare) { + this.reference = toCompare; + } + + @Override + public boolean equalToReference(KV candidate) { + K key = reference.getKey(); + K otherKey = candidate.getKey(); + if (key == null && otherKey == null) { + return true; + } else if(key == null || otherKey == null) { + return false; + } else { + return key.equals(otherKey); + } + } + + @Override + public int compareToReference(TypeComparator> other) { + return compare(this.reference, ((KvCoderComperator) other).reference); + } + + @Override + public int compare(KV first, KV second) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ByteArrayOutputStream baosOther = new ByteArrayOutputStream(); + try { + coder.getKeyCoder().encode(first.getKey(), baos, Coder.Context.OUTER); + coder.getKeyCoder().encode(second.getKey(), baosOther, Coder.Context.OUTER); + byte[] arr = baos.toByteArray(); + byte[] arrOther = baosOther.toByteArray(); + int len = arr.length < arrOther.length ? arr.length : arrOther.length; + for(int i = 0; i < len; i++ ) { + if (arr[i] != arrOther[i]) { + return arr[i] - arrOther[i]; + } + } + return arr.length - arrOther.length; + } catch (IOException e) { + throw new RuntimeException("Could not compare reference.", e); + } + } + + @Override + public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { + CoderTypeSerializer> serializer = new CoderTypeSerializer>(coder); + KV first = serializer.deserialize(firstSource); + KV second = serializer.deserialize(secondSource); + //K keyFirst = first.getKey(); + //K keySecond = first.getKey(); + return compare(first, second); + } + + @Override + public boolean supportsNormalizedKey() { + return false; + } + + @Override + public boolean supportsSerializationWithKeyNormalization() { + return false; + } + + @Override + public int getNormalizeKeyLen() { + return 0; + } + + @Override + public boolean isNormalizedKeyPrefixOnly(int keyBytes) { + return false; + } + + @Override + public void putNormalizedKey(KV record, MemorySegment target, int offset, int numBytes) { + + } + + @Override + public void writeWithKeyNormalization(KV record, DataOutputView target) throws IOException { + + } + + @Override + public KV readWithKeyDenormalization(KV reuse, DataInputView source) throws IOException { + return null; + } + + @Override + public boolean invertNormalizedKey() { + return false; + } + + @Override + public TypeComparator> duplicate() { + return new KvCoderComperator<>(coder); + } + + @Override + public int extractKeys(Object record, Object[] target, int index) { + return 0; + } + + @Override + public TypeComparator[] getFlatComparators() { + return new TypeComparator[0]; + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TransformToFlinkOpTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TransformTranslator.java similarity index 77% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TransformToFlinkOpTranslator.java rename to runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TransformTranslator.java index 303fab274397..c7c4bcb56fc8 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TransformToFlinkOpTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TransformTranslator.java @@ -4,7 +4,7 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; -public interface TransformToFlinkOpTranslator> { +public interface TransformTranslator> { void translateNode(TransformTreeNode node, Type transform, TranslationContext context); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index 7575b0426f10..deb03cad1cb2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -1,9 +1,13 @@ package com.dataartisans.flink.dataflow.translation; -import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.cloud.dataflow.sdk.values.TypedPValue; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.typeutils.GenericTypeInfo; import java.util.HashMap; import java.util.Map; @@ -11,6 +15,7 @@ public class TranslationContext { private final Map> dataSets; + private final Map, DataSet> broadcastDataSets; private final ExecutionEnvironment env; @@ -19,6 +24,7 @@ public class TranslationContext { public TranslationContext(ExecutionEnvironment env) { this.env = env; this.dataSets = new HashMap<>(); + this.broadcastDataSets = new HashMap<>(); } // ------------------------------------------------------------------------ @@ -27,30 +33,34 @@ public ExecutionEnvironment getExecutionEnvironment() { return env; } -// private DataSet getDataSet(TransformTreeNode node) { -// @SuppressWarnings("unchecked") -// DataSet typedSet = (DataSet) dataSets.get(node); -// return typedSet; -// } - - public DataSet getInputDataSet(TransformTreeNode node) { - PValue value = (PValue) node.getInput(); - return dataSets.get(value); + @SuppressWarnings("unchecked") + public DataSet getInputDataSet(PValue value) { + return (DataSet) dataSets.get(value); } - public void setOutputDataSet(TransformTreeNode node, DataSet value) { - PValue output = (PValue) node.getOutput(); - dataSets.put(output, value); + public void setOutputDataSet(PValue value, DataSet set) { + if (!dataSets.containsKey(value)) { + dataSets.put(value, set); + } + } + + @SuppressWarnings("unchecked") + public DataSet getSideInputDataSet(PCollectionView value) { + return (DataSet) broadcastDataSets.get(value); + } + + public void setSideInputDataSet(PCollectionView value, DataSet set) { + if (!broadcastDataSets.containsKey(value)) { + broadcastDataSets.put(value, set); + } } -// public void registerDataSet(DataSet dataSet, TransformTreeNode node) { -// DataSet previous = dataSets.put(node, dataSet); -// -// if (previous != null) { -// // undo the action -// dataSets.put(node, previous); -// throw new IllegalArgumentException( -// "Context contains already a DataSet as the result of the given TreeTransformNode."); -// } -// } + @SuppressWarnings("unchecked") + public TypeInformation getTypeInfo(PValue output) { + if (output instanceof TypedPValue) { + Coder outputCoder = ((TypedPValue) output).getCoder(); + return new CoderTypeInformation(outputCoder); + } + return new GenericTypeInfo((Class)Object.class); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/CombineFnAggregatorWrapper.java new file mode 100644 index 000000000000..9c613e20a167 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/CombineFnAggregatorWrapper.java @@ -0,0 +1,74 @@ +package com.dataartisans.flink.dataflow.translation.utils; + +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.common.collect.Lists; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.io.Serializable; + + +public class CombineFnAggregatorWrapper implements Aggregator, Accumulator { + + private AA aa; + private Combine.CombineFn combiner; + + public CombineFnAggregatorWrapper() { + } + + public CombineFnAggregatorWrapper(Combine.CombineFn combiner) { + this.combiner = combiner; + this.aa = combiner.createAccumulator(); + } + + @Override + public void add(AI value) { + combiner.addInput(aa, value); + } + + @Override + public AR getLocalValue() { + return combiner.extractOutput(aa); + } + + @Override + public void resetLocal() { + aa = combiner.createAccumulator(); + } + + @Override + @SuppressWarnings("unchecked") + public void merge(Accumulator other) { + aa = combiner.mergeAccumulators(Lists.newArrayList(aa, ((CombineFnAggregatorWrapper)other).aa)); + } + + @Override + public void addValue(AI value) { + add(value); + } + + @Override + public void write(DataOutputView out) throws IOException { + byte[] aaByte = SerializableUtils.serializeToByteArray((Serializable) aa); + byte[] combinerByte = SerializableUtils.serializeToByteArray(combiner); + out.write(aaByte.length); + out.write(aaByte); + out.write(combinerByte.length); + out.write(combinerByte); + } + + @Override + @SuppressWarnings("unchecked") + public void read(DataInputView in) throws IOException { + byte[] aaByte = new byte[in.readInt()]; + in.read(aaByte); + byte[] combinerByte = new byte[in.readInt()]; + in.read(combinerByte); + this.aa = (AA) SerializableUtils.deserializeFromByteArray(aaByte, "AggreatorValue"); + this.combiner = (Combine.CombineFn) SerializableUtils.deserializeFromByteArray(combinerByte, "AggreatorCombiner"); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/DataInputStreamWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/DataInputStreamWrapper.java new file mode 100644 index 000000000000..e5524ddbf079 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/DataInputStreamWrapper.java @@ -0,0 +1,29 @@ +package com.dataartisans.flink.dataflow.translation.utils; + +import org.apache.flink.core.memory.DataInputView; + +import java.io.IOException; +import java.io.InputStream; + +public class DataInputStreamWrapper extends InputStream { + + private DataInputView inputView; + + public DataInputStreamWrapper(DataInputView inputView) { + this.inputView = inputView; + } + + public void setInputView(DataInputView inputView) { + this.inputView = inputView; + } + + @Override + public int read() throws IOException { + return inputView.readByte(); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + return inputView.read(b, off, len); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/DataOutputStreamWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/DataOutputStreamWrapper.java new file mode 100644 index 000000000000..f9cc5f89c0ea --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/DataOutputStreamWrapper.java @@ -0,0 +1,29 @@ +package com.dataartisans.flink.dataflow.translation.utils; + +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.io.OutputStream; + +public class DataOutputStreamWrapper extends OutputStream { + + private DataOutputView outputView; + + public DataOutputStreamWrapper(DataOutputView outputView) { + this.outputView = outputView; + } + + public void setOutputView(DataOutputView outputView) { + this.outputView = outputView; + } + + @Override + public void write(int b) throws IOException { + outputView.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + outputView.write(b, off, len); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/SerializableFnAggregatorWrapper.java new file mode 100644 index 000000000000..b3d08334ed8f --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/utils/SerializableFnAggregatorWrapper.java @@ -0,0 +1,76 @@ +package com.dataartisans.flink.dataflow.translation.utils; + +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.common.collect.ImmutableList; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.io.Serializable; + + +public class SerializableFnAggregatorWrapper implements Aggregator, Accumulator { + + private AO result; + private SerializableFunction, AO> serFun; + + public SerializableFnAggregatorWrapper() { + } + + public SerializableFnAggregatorWrapper(SerializableFunction, AO> serFun) { + this.serFun = serFun; + resetLocal(); + } + + @Override + @SuppressWarnings("unchecked") + public void add(AI value) { + this.result = serFun.apply(ImmutableList.of((AI) result, value)); + } + + @Override + public AO getLocalValue() { + return result; + } + + @Override + public void resetLocal() { + this.result = serFun.apply(ImmutableList.of()); + } + + @Override + @SuppressWarnings("unchecked") + public void merge(Accumulator other) { + this.result = serFun.apply(ImmutableList.of((AI) result, (AI) other.getLocalValue())); + } + + @Override + public void addValue(AI value) { + add(value); + } + + @Override + public void write(DataOutputView out) throws IOException { + byte[] aaByte = SerializableUtils.serializeToByteArray((Serializable) result); + byte[] combinerByte = SerializableUtils.serializeToByteArray(serFun); + out.writeInt(aaByte.length); + out.write(aaByte); + out.writeInt(combinerByte.length); + out.write(combinerByte); + } + + @Override + @SuppressWarnings("unchecked") + public void read(DataInputView in) throws IOException { + byte[] aaByte = new byte[in.readInt()]; + in.read(aaByte); + byte[] serFunByte = new byte[in.readInt()]; + in.read(serFunByte); + this.result = (AO) SerializableUtils.deserializeFromByteArray(aaByte, "AggreatorValue"); + this.serFun = (SerializableFunction, AO>) SerializableUtils.deserializeFromByteArray(serFunByte, "AggreatorSerializableFunction"); + + } +} From ce0137b9b9f3fb92282302d27ae290d9b7a2339a Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 18 Feb 2015 17:45:39 +0100 Subject: [PATCH 009/149] Add WordCountITCase --- runners/flink/pom.xml | 8 +- .../dataflow/FlinkLocalPipelineRunner.java | 49 ++++++++-- .../flink/dataflow/FlinkPipelineOptions.java | 34 +++++++ .../FlinkTransformTranslators.java | 34 ++++--- .../flink/dataflow/FlinkTestPipeline.java | 94 +++++++++++++++++++ .../flink/dataflow/WordCountITCase.java | 75 +++++++++++++++ 6 files changed, 272 insertions(+), 22 deletions(-) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 5911ad6c02e9..eb683c236eed 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -1,6 +1,6 @@ org.apache.maven.plugins @@ -181,50 +181,31 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - [2.12.1,) - - check - - - - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - [1.0.0,) - - enforce - - - - - - - - - - - - + + + + org.apache.maven.plugins + maven-shade-plugin + 2.3 + + + package + + shade + + + + + com.google + flink.dataflow.shaded.com.google + + + + + + + diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/GoogleWordCountExampleRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/GoogleWordCountExampleRunner.java deleted file mode 100644 index 01eeb23094da..000000000000 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/GoogleWordCountExampleRunner.java +++ /dev/null @@ -1,31 +0,0 @@ -package com.dataartisans.flink.dataflow.examples; - -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.google.cloud.dataflow.examples.WordCount.CountWords; -import com.google.cloud.dataflow.examples.WordCount.Options; -import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; - -public class GoogleWordCountExampleRunner { - - public static void main(String[] args) { - - Options options = PipelineOptionsFactory.create().as(Options.class); - options.setOutput("/tmp/output2.txt"); - options.setInput("/tmp/documents/hello_world.txt"); - //options.setRunner(DirectPipelineRunner.class); - options.setRunner(FlinkPipelineRunner.class); - - Pipeline p = Pipeline.create(options); - - p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) - .apply(new CountWords()) - .apply(TextIO.Write.named("WriteCounts") - .to(options.getOutput()) - .withNumShards(options.getNumShards())); - - p.run(); - } - -} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java new file mode 100644 index 000000000000..4981350e3ef4 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java @@ -0,0 +1,55 @@ +package com.dataartisans.flink.dataflow.examples; + +import com.dataartisans.flink.dataflow.FlinkPipelineOptions; +import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +import com.google.cloud.dataflow.examples.WordCount.CountWords; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; + +public class WordCount { + + /** + * Options supported by {@link WordCount}. + *

+ * Inherits standard configuration options. + */ + public static interface Options extends PipelineOptions, FlinkPipelineOptions { + @Description("Path of the file to read from") + @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + String getInput(); + void setInput(String value); + + @Description("Path of the file to write to") + String getOutput(); + void setOutput(String value); + + /** + * By default (numShards == 0), the system will choose the shard count. + * Most programs will not need this option. + */ + @Description("Number of output shards (0 if the system should choose automatically)") + int getNumShards(); + void setNumShards(int value); + } + + public static void main(String[] args) { + + Options options = PipelineOptionsFactory.fromArgs(args).as(Options.class); + options.setRunner(FlinkPipelineRunner.class); + + Pipeline p = Pipeline.create(options); + + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(new CountWords()) + .apply(TextIO.Write.named("WriteCounts") + .to(options.getOutput()) + .withNumShards(options.getNumShards())); + + p.run(); + } + +} From 2e8d57d8b77c915b1d8a2c9e908f0c47a25fb93a Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 26 Feb 2015 14:21:47 +0100 Subject: [PATCH 043/149] Fix CoderComparators to correctly compare key lengths --- .../translation/types/CoderComperator.java | 12 ++++++++---- .../translation/types/KvCoderComperator.java | 17 +++++++++++------ 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java index c8d62a489f45..80bf41c64722 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java @@ -65,13 +65,16 @@ public int compare(T first, T second) { coder.encode(second, byteBuffer2, Coder.Context.OUTER); byte[] arr = byteBuffer1.getBuffer(); byte[] arrOther = byteBuffer2.getBuffer(); - int len = Math.min(byteBuffer1.size(), byteBuffer2.size()); + if (byteBuffer1.size() != byteBuffer2.size()) { + return arr.length - arrOther.length; + } + int len = byteBuffer1.size(); for(int i = 0; i < len; i++ ) { if (arr[i] != arrOther[i]) { return arr[i] - arrOther[i]; } } - return arr.length - arrOther.length; + return 0; } catch (IOException e) { throw new RuntimeException("Could not compare reference.", e); } @@ -132,11 +135,12 @@ public TypeComparator duplicate() { @Override public int extractKeys(Object record, Object[] target, int index) { - return 0; + target[index] = record; + return 1; } @Override public TypeComparator[] getFlatComparators() { - return new TypeComparator[0]; + return new TypeComparator[] { this.duplicate() }; } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index 9d349d41bda8..789b16d2c0e0 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -30,7 +30,6 @@ public class KvCoderComperator extends TypeComparator> { // For deserializing the key private transient DataInputViewWrapper inputWrapper; - public KvCoderComperator(KvCoder coder) { this.coder = coder; @@ -95,13 +94,16 @@ public int compare(KV first, KV second) { coder.getKeyCoder().encode(second.getKey(), byteBuffer2, Coder.Context.OUTER); byte[] arr = byteBuffer1.getBuffer(); byte[] arrOther = byteBuffer2.getBuffer(); - int len = Math.min(byteBuffer1.size(), byteBuffer2.size()); + if (byteBuffer1.size() != byteBuffer2.size()) { + return byteBuffer1.size() - byteBuffer2.size(); + } + int len = byteBuffer1.size(); for(int i = 0; i < len; i++ ) { if (arr[i] != arrOther[i]) { return arr[i] - arrOther[i]; } } - return arr.length - arrOther.length; + return 0; } catch (IOException e) { throw new RuntimeException("Could not compare reference.", e); } @@ -122,13 +124,16 @@ public int compareSerialized(DataInputView firstSource, DataInputView secondSour coder.getKeyCoder().encode(secondKey, byteBuffer2, Coder.Context.OUTER); byte[] arr = byteBuffer1.getBuffer(); byte[] arrOther = byteBuffer2.getBuffer(); - int len = Math.min(byteBuffer1.size(), byteBuffer2.size()); + if (byteBuffer1.size() != byteBuffer2.size()) { + return byteBuffer1.size() - byteBuffer2.size(); + } + int len = byteBuffer1.size(); for(int i = 0; i < len; i++ ) { if (arr[i] != arrOther[i]) { return arr[i] - arrOther[i]; } } - return arr.length - arrOther.length; + return 0; } catch (IOException e) { throw new RuntimeException("Could not compare reference.", e); } @@ -189,6 +194,6 @@ public int extractKeys(Object record, Object[] target, int index) { @Override public TypeComparator[] getFlatComparators() { - return new TypeComparator[] { new CoderComperator<>(coder.getKeyCoder())}; + return new TypeComparator[] {new CoderComperator<>(coder.getKeyCoder())}; } } From 12fb5431aa4265a90a760dccfe98b00b8f9cc2ba Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 26 Feb 2015 14:22:06 +0100 Subject: [PATCH 044/149] Add dependency-reduced-pom.xml to .gitignore --- runners/flink/.gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/runners/flink/.gitignore b/runners/flink/.gitignore index f9b86224eb52..218a1185da17 100644 --- a/runners/flink/.gitignore +++ b/runners/flink/.gitignore @@ -1,3 +1,4 @@ +dependency-reduced-pom.xml .cache .classpath .idea From b7e4907028367ee41d8f5a568b7b030ac7e0bc3d Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 26 Feb 2015 14:22:51 +0100 Subject: [PATCH 045/149] Correctly forward EOFExceptions from CoderTypeSerializer --- .../translation/types/CoderTypeSerializer.java | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java index 53f68368e6c6..9ab92faab8db 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java @@ -3,11 +3,13 @@ import com.dataartisans.flink.dataflow.translation.wrappers.DataInputViewWrapper; import com.dataartisans.flink.dataflow.translation.wrappers.DataOutputViewWrapper; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import java.io.ByteArrayInputStream; +import java.io.EOFException; import java.io.IOException; import java.io.ObjectInputStream; @@ -47,7 +49,7 @@ public boolean isImmutableType() { @Override public boolean isStateful() { - return false; + return true; } @Override @@ -90,8 +92,17 @@ public void serialize(T t, DataOutputView dataOutputView) throws IOException { @Override public T deserialize(DataInputView dataInputView) throws IOException { - inputWrapper.setInputView(dataInputView); - return coder.decode(inputWrapper, Coder.Context.NESTED); + try { + inputWrapper.setInputView(dataInputView); + return coder.decode(inputWrapper, Coder.Context.NESTED); + } catch (CoderException e) { + Throwable cause = e.getCause(); + if (cause instanceof EOFException) { + throw (EOFException) cause; + } else { + throw e; + } + } } @Override From d5ac62471821f7d4fe637bbd10bc2350dfc86469 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 26 Feb 2015 14:23:32 +0100 Subject: [PATCH 046/149] Fix mismatch between DataInput and InputStream in DataInputViewWrapper --- .../translation/wrappers/DataInputViewWrapper.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java index a752c85ba8d1..d311bf7f169a 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java @@ -2,6 +2,7 @@ import org.apache.flink.core.memory.DataInputView; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; @@ -25,7 +26,13 @@ public void setInputView(DataInputView inputView) { @Override public int read() throws IOException { - return inputView.readByte(); + try { + return inputView.readUnsignedByte(); + } catch (EOFException e) { + // translate between DataInput and InputStream, + // DataInput signals EOF by exception, InputStream does it by returning -1 + return -1; + } } @Override From 83177ad0b2a97e17f7907a64d601fffd63ff157e Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 26 Feb 2015 14:25:30 +0100 Subject: [PATCH 047/149] Add Fake Grouping Key for KV GroupByKey translation --- .../dataflow/translation/FlinkTransformTranslators.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 335ac628e88a..e863c923b092 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -250,7 +250,7 @@ public void translateNode(GroupByKey.GroupByKeyOnly transform, Translation TypeInformation>> typeInformation = context.getTypeInfo(transform.getOutput()); - Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType())); + Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); GroupReduceOperator, KV>> outputDataSet = new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); @@ -279,7 +279,7 @@ public void translateNode(Combine.PerKey transform, TranslationContex TypeInformation> typeInformation = context.getTypeInfo(transform.getOutput()); - Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType())); + Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); GroupReduceOperator, KV> outputDataSet = new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); @@ -484,8 +484,8 @@ private static void transformSideInputs(List> sideInputs, // // CoGroupKeyedListAggregator aggregator = new CoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2); // -// ExpressionKeys> keySelector1 = new ExpressionKeys<>(new String[]{""}, inputDataSet1.getType()); -// ExpressionKeys> keySelector2 = new ExpressionKeys<>(new String[]{""}, inputDataSet2.getType()); +// ExpressionKeys> keySelector1 = new ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType()); +// ExpressionKeys> keySelector2 = new ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType()); // // DataSet> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2, // keySelector1, keySelector2, From 61238d33ab53cd83d7d8533ad866ff49ac0b8e6d Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 26 Feb 2015 15:50:44 +0100 Subject: [PATCH 048/149] Change Flink Grouping Reducer to not materialize values --- .../FlinkKeyedListAggregationFunction.java | 47 ++++++++++++++++--- 1 file changed, 40 insertions(+), 7 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java index 81f22eb04dc7..8b5766557c05 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java @@ -5,6 +5,7 @@ import org.apache.flink.util.Collector; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; /** @@ -17,13 +18,45 @@ public class FlinkKeyedListAggregationFunction implements GroupReduceFuncti @Override public void reduce(Iterable> values, Collector>> out) throws Exception { - K k = null; - List result = new ArrayList(); - for (KV kv : values) { - k = kv.getKey(); - result.add(kv.getValue()); - } - out.collect(KV.of(k, (Iterable) result)); + Iterator> it = values.iterator(); + KV first = it.next(); + Iterable passThrough = new PassThroughIterable<>(first, it); + out.collect(KV.of(first.getKey(), passThrough)); } + private static class PassThroughIterable implements Iterable, Iterator { + private KV first; + private Iterator> iterator; + + public PassThroughIterable(KV first, Iterator> iterator) { + this.first = first; + this.iterator = iterator; + } + + @Override + public Iterator iterator() { + return this; + } + + @Override + public boolean hasNext() { + return first != null || iterator.hasNext(); + } + + @Override + public V next() { + if (first != null) { + V result = first.getValue(); + first = null; + return result; + } else { + return iterator.next().getValue(); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Cannot remove elements from input."); + } + } } From 7b502364b57ecde3b7365d9ae0650fc80e23edcc Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 26 Feb 2015 16:16:32 +0100 Subject: [PATCH 049/149] Add Partial Reduce Translation using MapPartition --- .../FlinkTransformTranslators.java | 28 ++++++++--- .../functions/FlinkPartialReduceFunction.java | 50 +++++++++++++++++++ ...Function.java => FlinkReduceFunction.java} | 4 +- 3 files changed, 74 insertions(+), 8 deletions(-) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java rename runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/{FlinkCombineFunction.java => FlinkReduceFunction.java} (87%) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index e863c923b092..3fada5a31c14 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -1,7 +1,8 @@ package com.dataartisans.flink.dataflow.translation; import com.dataartisans.flink.dataflow.io.ConsoleIO; -import com.dataartisans.flink.dataflow.translation.functions.FlinkCombineFunction; +import com.dataartisans.flink.dataflow.translation.functions.FlinkPartialReduceFunction; +import com.dataartisans.flink.dataflow.translation.functions.FlinkReduceFunction; import com.dataartisans.flink.dataflow.translation.functions.FlinkCreateFunction; import com.dataartisans.flink.dataflow.translation.functions.FlinkDoFnFunction; import com.dataartisans.flink.dataflow.translation.functions.FlinkKeyedListAggregationFunction; @@ -13,6 +14,7 @@ import com.dataartisans.flink.dataflow.translation.wrappers.SourceInputFormat; import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.io.AvroIO; import com.google.cloud.dataflow.sdk.io.ReadSource; import com.google.cloud.dataflow.sdk.io.Source; @@ -264,25 +266,39 @@ private static class CombinePerKeyTranslator implements FlinkPipeline public void translateNode(Combine.PerKey transform, TranslationContext context) { DataSet> inputDataSet = context.getInputDataSet(transform.getInput()); - Combine.KeyedCombineFn keyedCombineFn = null; + Combine.KeyedCombineFn keyedCombineFn = null; // This is super hacky, but unfortunately we cannot get the fn otherwise try { Field fnField = transform.getClass().getDeclaredField("fn"); fnField.setAccessible(true); - keyedCombineFn = (Combine.KeyedCombineFn) fnField.get(transform); + keyedCombineFn = (Combine.KeyedCombineFn) fnField.get(transform); } catch (NoSuchFieldException | IllegalAccessException e) { // we know that the field is there and it is accessible System.out.println("Could not access KeyedCombineFn: " + e); } - GroupReduceFunction, KV> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn); + KvCoder inputCoder = (KvCoder) transform.getInput().getCoder(); - TypeInformation> typeInformation = context.getTypeInfo(transform.getOutput()); + Coder accumulatorCoder = + keyedCombineFn.getAccumulatorCoder(transform.getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder()); + + TypeInformation> partialReduceTypeInfo = new CoderTypeInformation<>(accumulatorCoder); + + + FlinkPartialReduceFunction partialReduceFunction = new + FlinkPartialReduceFunction<>(keyedCombineFn); + + + + + GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); + + TypeInformation> reduceTypeInfo = context.getTypeInfo(transform.getOutput()); Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); GroupReduceOperator, KV> outputDataSet = - new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); + new GroupReduceOperator<>(grouping, reduceTypeInfo, reduceFunction, transform.getName()); context.setOutputDataSet(transform.getOutput(), outputDataSet); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java new file mode 100644 index 000000000000..e8264aef15ee --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java @@ -0,0 +1,50 @@ +package com.dataartisans.flink.dataflow.translation.functions; + +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.values.KV; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.MapPartitionFunction; +import org.apache.flink.util.Collector; + +import java.util.Iterator; + +/** + * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a + * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} operation. This reads the input + * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and collects + * the values in a {@code List}. + */ +public class FlinkPartialReduceFunction implements MapPartitionFunction, KV> { + + private final Combine.KeyedCombineFn keyedCombineFn; + + public FlinkPartialReduceFunction(Combine.KeyedCombineFn + keyedCombineFn) { + this.keyedCombineFn = keyedCombineFn; + } + + @Override + public void mapPartition(Iterable> values, Collector> out) throws + Exception { + Iterator> it = values.iterator(); + KV current = it.next(); + K currentKey = current.getKey(); + VA accumulator = keyedCombineFn.createAccumulator(currentKey); + keyedCombineFn.addInput(currentKey, accumulator, current.getValue()); + + while (it.hasNext()) { + current = it.next(); + if (current.getKey().equals(currentKey)) { + keyedCombineFn.addInput(currentKey, accumulator, current.getValue()); + } else { + // output current accumulation value, start new accumulation + out.collect(KV.of(currentKey, accumulator)); + currentKey = current.getKey(); + accumulator = keyedCombineFn.createAccumulator(currentKey); + } + } + + // also emit last accumulation + out.collect(KV.of(currentKey, accumulator)); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCombineFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java similarity index 87% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCombineFunction.java rename to runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java index cebbdf3470d4..dda349625987 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCombineFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java @@ -13,11 +13,11 @@ * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and collects * the values in a {@code List}. */ -public class FlinkCombineFunction implements GroupReduceFunction, KV> { +public class FlinkReduceFunction implements GroupReduceFunction, KV> { private final Combine.KeyedCombineFn keyedCombineFn; - public FlinkCombineFunction(Combine.KeyedCombineFn keyedCombineFn) { + public FlinkReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { this.keyedCombineFn = (Combine.KeyedCombineFn) keyedCombineFn; } From 3a9115e7f9beb25b261e03f982e7c48792127643 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 26 Feb 2015 17:48:49 +0100 Subject: [PATCH 050/149] Move to Flink 0.9-SNAPSHOT add Combine based on MapPartition --- runners/flink/pom.xml | 16 +- .../FlinkTransformTranslators.java | 28 ++-- .../translation/TranslationContext.java | 8 +- .../functions/FlinkPartialReduceFunction.java | 18 +- .../functions/FlinkReduceFunction.java | 29 ++-- .../types/CoderTypeInformation.java | 45 +---- .../types/CoderTypeSerializer.java | 4 +- .../types/KvCoderTypeInformation.java | 155 ++++++++++++++++++ .../types/VoidCoderTypeSerializer.java | 4 +- .../wrappers/CombineFnAggregatorWrapper.java | 26 ++- .../SerializableFnAggregatorWrapper.java | 40 +++-- .../wrappers/SourceInputFormat.java | 2 +- 12 files changed, 282 insertions(+), 93 deletions(-) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 0b851f6df02d..ede8f5c20976 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -39,9 +39,23 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans UTF-8 UTF-8 - 0.8.0 + 0.9-SNAPSHOT + + + apache.snapshots + Apache Development Snapshot Repository + https://repository.apache.org/content/repositories/snapshots/ + + false + + + true + + + + org.apache.flink diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 3fada5a31c14..675debadf835 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -11,6 +11,7 @@ import com.dataartisans.flink.dataflow.translation.functions.RawUnionValue; import com.dataartisans.flink.dataflow.translation.functions.UnionCoder; import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; +import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation; import com.dataartisans.flink.dataflow.translation.wrappers.SourceInputFormat; import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -34,8 +35,10 @@ import com.google.common.collect.Lists; import org.apache.avro.Schema; import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.SortPartitionOperator; import org.apache.flink.api.java.io.AvroInputFormat; import org.apache.flink.api.java.io.AvroOutputFormat; import org.apache.flink.api.java.io.TextInputFormat; @@ -260,18 +263,18 @@ public void translateNode(GroupByKey.GroupByKeyOnly transform, Translation } } - private static class CombinePerKeyTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class CombinePerKeyTranslator implements FlinkPipelineTranslator.TransformTranslator> { @Override public void translateNode(Combine.PerKey transform, TranslationContext context) { DataSet> inputDataSet = context.getInputDataSet(transform.getInput()); - Combine.KeyedCombineFn keyedCombineFn = null; + Combine.KeyedCombineFn keyedCombineFn = null; // This is super hacky, but unfortunately we cannot get the fn otherwise try { Field fnField = transform.getClass().getDeclaredField("fn"); fnField.setAccessible(true); - keyedCombineFn = (Combine.KeyedCombineFn) fnField.get(transform); + keyedCombineFn = (Combine.KeyedCombineFn) fnField.get(transform); } catch (NoSuchFieldException | IllegalAccessException e) { // we know that the field is there and it is accessible System.out.println("Could not access KeyedCombineFn: " + e); @@ -279,25 +282,30 @@ public void translateNode(Combine.PerKey transform, TranslationContex KvCoder inputCoder = (KvCoder) transform.getInput().getCoder(); - Coder accumulatorCoder = + Coder accumulatorCoder = keyedCombineFn.getAccumulatorCoder(transform.getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder()); - TypeInformation> partialReduceTypeInfo = new CoderTypeInformation<>(accumulatorCoder); + TypeInformation> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder)); - FlinkPartialReduceFunction partialReduceFunction = new - FlinkPartialReduceFunction<>(keyedCombineFn); + FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); + SortPartitionOperator> sortPartitionOperator = new SortPartitionOperator<> + (inputDataSet, "key", Order.ASCENDING, "Sort for PartialReduce: " + + transform.getName()); + MapPartitionOperator, KV> partialReduceOperator = new + MapPartitionOperator<>(sortPartitionOperator, partialReduceTypeInfo, + partialReduceFunction, "PartialReduce: " + transform.getName()); - GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); + GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); TypeInformation> reduceTypeInfo = context.getTypeInfo(transform.getOutput()); - Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); + Grouping> grouping = new UnsortedGrouping<>(partialReduceOperator, new Keys.ExpressionKeys<>(new String[]{"key"}, partialReduceOperator.getType())); - GroupReduceOperator, KV> outputDataSet = + GroupReduceOperator, KV> outputDataSet = new GroupReduceOperator<>(grouping, reduceTypeInfo, reduceFunction, transform.getName()); context.setOutputDataSet(transform.getOutput(), outputDataSet); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index 04b44ec54e21..400c97a81b97 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -1,7 +1,9 @@ package com.dataartisans.flink.dataflow.translation; import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; +import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.PValue; @@ -67,7 +69,11 @@ public void setSideInputDataSet(PCollectionView value, DataSet set) { public TypeInformation getTypeInfo(PValue output) { if (output instanceof TypedPValue) { Coder outputCoder = ((TypedPValue) output).getCoder(); - return new CoderTypeInformation(outputCoder); + if (outputCoder instanceof KvCoder) { + return new KvCoderTypeInformation((KvCoder) outputCoder); + } else { + return new CoderTypeInformation(outputCoder); + } } return new GenericTypeInfo((Class)Object.class); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java index e8264aef15ee..0b1efb28a2b3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java @@ -9,10 +9,13 @@ import java.util.Iterator; /** - * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a - * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} operation. This reads the input - * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and collects - * the values in a {@code List}. + * Flink {@link org.apache.flink.api.common.functions.MapPartitionFunction} for executing a + * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input + * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and emits accumulated + * values. + * + * This assumes that the input values are sorted. We basically implement a GroupReduce operation + * inside a partition here. */ public class FlinkPartialReduceFunction implements MapPartitionFunction, KV> { @@ -27,6 +30,10 @@ public FlinkPartialReduceFunction(Combine.KeyedCombineFn public void mapPartition(Iterable> values, Collector> out) throws Exception { Iterator> it = values.iterator(); + if (!it.hasNext()) { + return; + } + KV current = it.next(); K currentKey = current.getKey(); VA accumulator = keyedCombineFn.createAccumulator(currentKey); @@ -34,13 +41,14 @@ public void mapPartition(Iterable> values, Collector> out) t while (it.hasNext()) { current = it.next(); - if (current.getKey().equals(currentKey)) { + if (currentKey == null && current.getKey() == null || current.getKey().equals(currentKey)) { keyedCombineFn.addInput(currentKey, accumulator, current.getValue()); } else { // output current accumulation value, start new accumulation out.collect(KV.of(currentKey, accumulator)); currentKey = current.getKey(); accumulator = keyedCombineFn.createAccumulator(currentKey); + keyedCombineFn.addInput(currentKey, accumulator, current.getValue()); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java index dda349625987..9a12d1a8c4eb 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java @@ -2,6 +2,7 @@ import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.collect.ImmutableList; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.util.Collector; @@ -9,29 +10,29 @@ /** * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a - * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} operation. This reads the input - * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and collects - * the values in a {@code List}. + * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input + * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and merges the + * accumulators resulting from the PartialReduce in the MapPartition operation. */ -public class FlinkReduceFunction implements GroupReduceFunction, KV> { +public class FlinkReduceFunction implements GroupReduceFunction, KV> { - private final Combine.KeyedCombineFn keyedCombineFn; + private final Combine.KeyedCombineFn keyedCombineFn; - public FlinkReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { - this.keyedCombineFn = (Combine.KeyedCombineFn) keyedCombineFn; + public FlinkReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { + this.keyedCombineFn = keyedCombineFn; } @Override - public void reduce(Iterable> values, Collector> out) throws Exception { - Iterator> it = values.iterator(); - KV current = it.next(); - Object accumulator = keyedCombineFn.createAccumulator(current.getKey()); - keyedCombineFn.addInput(current.getKey(), accumulator, current.getValue()); + public void reduce(Iterable> values, Collector> out) throws Exception { + Iterator> it = values.iterator(); + KV current = it.next(); + K k = current.getKey(); + VA accumulator = current.getValue(); while (it.hasNext()) { current = it.next(); - keyedCombineFn.addInput(current.getKey(), accumulator, current.getValue()); + keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) ); } - out.collect(KV.of(current.getKey(), keyedCombineFn.extractOutput(current.getKey(), + out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator))); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java index fb607b779f64..3a0fdd5f7619 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java @@ -3,7 +3,8 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -14,28 +15,18 @@ /** * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for - * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s + * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s. */ -public class CoderTypeInformation extends CompositeType { +public class CoderTypeInformation extends TypeInformation implements AtomicType { private Coder coder; @SuppressWarnings("unchecked") public CoderTypeInformation(Coder coder) { - // We don't have the Class, so we have to pass null here. What a shame... - super(null); this.coder = coder; Preconditions.checkNotNull(coder); } - @SuppressWarnings("unchecked") - public TypeComparator createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset) { - if (!(coder instanceof KvCoder)) { - throw new RuntimeException("Coder " + coder + " is not a KvCoder."); - } - return new KvCoderComperator((KvCoder) coder); - } - @Override public boolean isBasicType() { return false; @@ -48,7 +39,7 @@ public boolean isTupleType() { @Override public int getArity() { - return 0; + return 1; } @Override @@ -65,7 +56,7 @@ public boolean isKeyType() { @Override @SuppressWarnings("unchecked") - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig config) { if (coder instanceof VoidCoder) { return (TypeSerializer) new VoidCoderTypeSerializer(); } @@ -77,13 +68,6 @@ public int getTotalFields() { return 2; } - @Override - @SuppressWarnings("unchecked") - public void getKey(String fieldExpression, int offset, List result) { - result.add(new FlatFieldDescriptor(0, BasicTypeInfo.INT_TYPE_INFO)); - } - - @Override public boolean equals(Object o) { if (this == o) return true; @@ -107,20 +91,9 @@ public String toString() { '}'; } - // These methods we only have because we need to fulfill CompositeTypeInfo requirements. - @Override - public TypeInformation getTypeAt(int pos) { - return null; - } - @Override - protected void initializeNewComparator(int localKeyCount) { } - - @Override - protected TypeComparator getNewComparator() { - return null; + public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig + executionConfig) { + return new CoderComperator<>(coder); } - - @Override - protected void addCompareField(int fieldId, TypeComparator comparator) { } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java index 9ab92faab8db..d3b980157ad2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java @@ -48,8 +48,8 @@ public boolean isImmutableType() { } @Override - public boolean isStateful() { - return true; + public CoderTypeSerializer duplicate() { + return new CoderTypeSerializer<>(coder); } @Override diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java new file mode 100644 index 000000000000..b77f7ebc29db --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java @@ -0,0 +1,155 @@ +package com.dataartisans.flink.dataflow.translation.types; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.values.KV; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.CompositeType; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.shaded.com.google.common.base.Preconditions; + +import java.util.List; + +/** + * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for + * Dataflow {@link com.google.cloud.dataflow.sdk.coders.KvCoder}. + */ +public class KvCoderTypeInformation extends CompositeType> { + + private KvCoder coder; + + @SuppressWarnings("unchecked") + public KvCoderTypeInformation(KvCoder coder) { + // We don't have the Class, so we have to pass null here. What a shame... + super(null); + this.coder = coder; + Preconditions.checkNotNull(coder); + } + + @Override + @SuppressWarnings("unchecked") + public TypeComparator> createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) { + return new KvCoderComperator((KvCoder) coder); + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 2; + } + + @Override + @SuppressWarnings("unchecked") + public Class> getTypeClass() { + return privateGetTypeClass(); + } + + @SuppressWarnings("unchecked") + private static Class privateGetTypeClass() { + return (Class) Object.class; + } + + @Override + public boolean isKeyType() { + return true; + } + + @Override + @SuppressWarnings("unchecked") + public TypeSerializer> createSerializer(ExecutionConfig config) { + return new CoderTypeSerializer<>(coder); + } + + @Override + public int getTotalFields() { + return 2; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + KvCoderTypeInformation that = (KvCoderTypeInformation) o; + + return coder.equals(that.coder); + + } + + @Override + public int hashCode() { + return coder.hashCode(); + } + + @Override + public String toString() { + return "CoderTypeInformation{" + + "coder=" + coder + + '}'; + } + + @Override + public TypeInformation getTypeAt(int pos) { + if (pos == 0) { + return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); + } else if (pos == 1) { + return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); + } else { + throw new RuntimeException("Invalid field position " + pos); + } + } + + @Override + public TypeInformation getTypeAt(String fieldExpression) { + if (fieldExpression.equals("key")) { + return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); + } else if (fieldExpression.equals("value")) { + return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); + } else { + throw new UnsupportedOperationException("Only KvCoder has fields."); + } + } + + @Override + public String[] getFieldNames() { + return new String[]{"key", "value"}; + } + + @Override + public int getFieldIndex(String fieldName) { + if (fieldName.equals("key")) { + return 0; + } else if (fieldName.equals("value")) { + return 1; + } else { + return -1; + } + } + + // These three we only have because we support CompositeType, we create our own comparator + // in createComparator. + @Override + protected void initializeNewComparator(int localKeyCount) {} + @Override + protected void addCompareField(int fieldId, TypeComparator comparator) {} + @Override + protected TypeComparator> getNewComparator(ExecutionConfig config) { return null; } + + @Override + public void getFlatFields(String fieldExpression, int offset, List result) { + CoderTypeInformation keyTypeInfo = new CoderTypeInformation<>(coder.getKeyCoder()); + result.add(new FlatFieldDescriptor(0, keyTypeInfo)); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java index ab7b361c0805..f4517d22f304 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java @@ -20,8 +20,8 @@ public boolean isImmutableType() { } @Override - public boolean isStateful() { - return false; + public VoidCoderTypeSerializer duplicate() { + return this; } @Override diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java index 240f1dfc4f5b..6cde53cc47e6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java @@ -9,6 +9,8 @@ import org.apache.flink.core.memory.DataOutputView; import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.io.Serializable; /** @@ -17,10 +19,10 @@ * the combine function as an aggregator in a {@link com.google.cloud.dataflow.sdk.transforms.ParDo} * operation. */ -public class CombineFnAggregatorWrapper implements Aggregator, Accumulator { +public class CombineFnAggregatorWrapper implements Aggregator, Accumulator { private AA aa; - private Combine.CombineFn combiner; + private Combine.CombineFn combiner; public CombineFnAggregatorWrapper() { } @@ -36,8 +38,8 @@ public void add(AI value) { } @Override - public AR getLocalValue() { - return combiner.extractOutput(aa); + public Serializable getLocalValue() { + return (Serializable) combiner.extractOutput(aa); } @Override @@ -47,17 +49,27 @@ public void resetLocal() { @Override @SuppressWarnings("unchecked") - public void merge(Accumulator other) { + public void merge(Accumulator other) { aa = combiner.mergeAccumulators(Lists.newArrayList(aa, ((CombineFnAggregatorWrapper)other).aa)); } + @Override + public Accumulator clone() { + // copy it by merging + AA aaCopy = combiner.mergeAccumulators(Lists.newArrayList(aa)); + CombineFnAggregatorWrapper result = new + CombineFnAggregatorWrapper<>(combiner); + result.aa = aaCopy; + return result; + } + @Override public void addValue(AI value) { add(value); } @Override - public void write(DataOutputView out) throws IOException { + public void write(ObjectOutputStream out) throws IOException { byte[] aaByte = SerializableUtils.serializeToByteArray((Serializable) aa); byte[] combinerByte = SerializableUtils.serializeToByteArray(combiner); out.write(aaByte.length); @@ -68,7 +80,7 @@ public void write(DataOutputView out) throws IOException { @Override @SuppressWarnings("unchecked") - public void read(DataInputView in) throws IOException { + public void read(ObjectInputStream in) throws IOException { byte[] aaByte = new byte[in.readInt()]; in.read(aaByte); byte[] combinerByte = new byte[in.readInt()]; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java index 3b3e3a620849..05640f6b30cf 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -4,11 +4,12 @@ import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import org.apache.flink.api.common.accumulators.Accumulator; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.io.Serializable; /** @@ -17,9 +18,9 @@ * the function as an aggregator in a {@link com.google.cloud.dataflow.sdk.transforms.ParDo} * operation. */ -public class SerializableFnAggregatorWrapper implements Aggregator, Accumulator { +public class SerializableFnAggregatorWrapper implements Aggregator, Accumulator { - private AO result; + private AO aa; private SerializableFunction, AO> serFun; public SerializableFnAggregatorWrapper() { @@ -33,23 +34,23 @@ public SerializableFnAggregatorWrapper(SerializableFunction, AO> se @Override @SuppressWarnings("unchecked") public void add(AI value) { - this.result = serFun.apply(ImmutableList.of((AI) result, value)); + this.aa = serFun.apply(ImmutableList.of((AI) aa, value)); } @Override - public AO getLocalValue() { - return result; + public Serializable getLocalValue() { + return (Serializable) aa; } @Override public void resetLocal() { - this.result = serFun.apply(ImmutableList.of()); + this.aa = serFun.apply(ImmutableList.of()); } @Override @SuppressWarnings("unchecked") - public void merge(Accumulator other) { - this.result = serFun.apply(ImmutableList.of((AI) result, (AI) other.getLocalValue())); + public void merge(Accumulator other) { + this.aa = serFun.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue())); } @Override @@ -58,8 +59,8 @@ public void addValue(AI value) { } @Override - public void write(DataOutputView out) throws IOException { - byte[] aaByte = SerializableUtils.serializeToByteArray((Serializable) result); + public void write(ObjectOutputStream out) throws IOException { + byte[] aaByte = SerializableUtils.serializeToByteArray((Serializable) aa); byte[] combinerByte = SerializableUtils.serializeToByteArray(serFun); out.writeInt(aaByte.length); out.write(aaByte); @@ -69,13 +70,24 @@ public void write(DataOutputView out) throws IOException { @Override @SuppressWarnings("unchecked") - public void read(DataInputView in) throws IOException { + public void read(ObjectInputStream in) throws IOException { byte[] aaByte = new byte[in.readInt()]; in.read(aaByte); byte[] serFunByte = new byte[in.readInt()]; in.read(serFunByte); - this.result = (AO) SerializableUtils.deserializeFromByteArray(aaByte, "AggreatorValue"); + this.aa = (AO) SerializableUtils.deserializeFromByteArray(aaByte, "AggreatorValue"); this.serFun = (SerializableFunction, AO>) SerializableUtils.deserializeFromByteArray(serFunByte, "AggreatorSerializableFunction"); } + + @Override + public Accumulator clone() { + // copy it by merging + AO resultCopy = serFun.apply(Lists.newArrayList((AI) aa)); + SerializableFnAggregatorWrapper result = new + SerializableFnAggregatorWrapper<>(serFun); + + result.aa = resultCopy; + return result; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java index a1726f03e794..87b511dfc093 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java @@ -136,7 +136,7 @@ public InputSplitAssigner getInputSplitAssigner(final SourceInputSplit[] sourceI private int index = 0; private final SourceInputSplit[] splits = sourceInputSplits; @Override - public InputSplit getNextInputSplit(String host) { + public InputSplit getNextInputSplit(String host, int taskId) { if (index < splits.length) { return splits[index++]; } else { From 2e533710e45a33ec6b8ac3d85fee01dd3650126f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 27 Feb 2015 09:49:44 +0100 Subject: [PATCH 051/149] Rename CoGroupKeyedListAggregator to match Naming Scheme --- ...ListAggregator.java => FlinkCoGroupKeyedListAggregator.java} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/{CoGroupKeyedListAggregator.java => FlinkCoGroupKeyedListAggregator.java} (90%) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/CoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java similarity index 90% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/CoGroupKeyedListAggregator.java rename to runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java index 5e206362dced..d524846a246f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/CoGroupKeyedListAggregator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java @@ -6,7 +6,7 @@ import org.apache.flink.util.Collector; -public class CoGroupKeyedListAggregator implements CoGroupFunction, KV, KV>{ +public class FlinkCoGroupKeyedListAggregator implements CoGroupFunction, KV, KV>{ // private CoGbkResultSchema schema; // private TupleTag tupleTag1; From cf508691feacaba0b664c8a2d31c6665e4d61697 Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 4 Mar 2015 10:20:38 +0100 Subject: [PATCH 052/149] enable CoGroupByKey translator which was disabled due to pending pull request --- .../FlinkTransformTranslators.java | 93 +++++++++---------- .../FlinkCoGroupKeyedListAggregator.java | 47 +++++----- 2 files changed, 70 insertions(+), 70 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 675debadf835..7f3738564f0b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -1,15 +1,7 @@ package com.dataartisans.flink.dataflow.translation; import com.dataartisans.flink.dataflow.io.ConsoleIO; -import com.dataartisans.flink.dataflow.translation.functions.FlinkPartialReduceFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkReduceFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkCreateFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkDoFnFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkKeyedListAggregationFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkMultiOutputDoFnFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkMultiOutputPruningFunction; -import com.dataartisans.flink.dataflow.translation.functions.RawUnionValue; -import com.dataartisans.flink.dataflow.translation.functions.UnionCoder; +import com.dataartisans.flink.dataflow.translation.functions.*; import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation; import com.dataartisans.flink.dataflow.translation.wrappers.SourceInputFormat; @@ -28,6 +20,10 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionView; @@ -100,8 +96,7 @@ public class FlinkTransformTranslators { // Flink-specific TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslator()); - // Disabled because it depends on a pending pull request to the DataFlowSDK - //TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslator()); + TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslator()); } @@ -479,44 +474,44 @@ private static void transformSideInputs(List> sideInputs, } // Disabled because it depends on a pending pull request to the DataFlowSDK -// /** -// * Special composite transform translator. Only called if the CoGroup is two dimensional. -// * @param -// */ -// private static class CoGroupByKeyTranslator implements FlinkPipelineTranslator.TransformTranslator> { -// -// @Override -// public void translateNode(CoGroupByKey transform, TranslationContext context) { -// KeyedPCollectionTuple input = transform.getInput(); -// -// CoGbkResultSchema schema = input.getCoGbkResultSchema(); -// List> keyedCollections = input.getKeyedCollections(); -// -// KeyedPCollectionTuple.TaggedKeyedPCollection taggedCollection1 = keyedCollections.get(0); -// KeyedPCollectionTuple.TaggedKeyedPCollection taggedCollection2 = keyedCollections.get(1); -// -// TupleTag tupleTag1 = taggedCollection1.getTupleTag(); -// TupleTag tupleTag2 = taggedCollection2.getTupleTag(); -// -// PCollection> collection1 = taggedCollection1.getCollection(); -// PCollection> collection2 = taggedCollection2.getCollection(); -// -// DataSet> inputDataSet1 = context.getInputDataSet(collection1); -// DataSet> inputDataSet2 = context.getInputDataSet(collection2); -// -// TypeInformation> typeInfo = context.getTypeInfo(transform.getOutput()); -// -// CoGroupKeyedListAggregator aggregator = new CoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2); -// -// ExpressionKeys> keySelector1 = new ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType()); -// ExpressionKeys> keySelector2 = new ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType()); -// -// DataSet> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2, -// keySelector1, keySelector2, -// aggregator, typeInfo, null, transform.getName()); -// context.setOutputDataSet(transform.getOutput(), out); -// } -// } + /** + * Special composite transform translator. Only called if the CoGroup is two dimensional. + * @param + */ + private static class CoGroupByKeyTranslator implements FlinkPipelineTranslator.TransformTranslator> { + + @Override + public void translateNode(CoGroupByKey transform, TranslationContext context) { + KeyedPCollectionTuple input = transform.getInput(); + + CoGbkResultSchema schema = input.getCoGbkResultSchema(); + List> keyedCollections = input.getKeyedCollections(); + + KeyedPCollectionTuple.TaggedKeyedPCollection taggedCollection1 = keyedCollections.get(0); + KeyedPCollectionTuple.TaggedKeyedPCollection taggedCollection2 = keyedCollections.get(1); + + TupleTag tupleTag1 = taggedCollection1.getTupleTag(); + TupleTag tupleTag2 = taggedCollection2.getTupleTag(); + + PCollection> collection1 = taggedCollection1.getCollection(); + PCollection> collection2 = taggedCollection2.getCollection(); + + DataSet> inputDataSet1 = context.getInputDataSet(collection1); + DataSet> inputDataSet2 = context.getInputDataSet(collection2); + + TypeInformation> typeInfo = context.getTypeInfo(transform.getOutput()); + + FlinkCoGroupKeyedListAggregator aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2); + + Keys.ExpressionKeys> keySelector1 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType()); + Keys.ExpressionKeys> keySelector2 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType()); + + DataSet> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2, + keySelector1, keySelector2, + aggregator, typeInfo, null, transform.getName()); + context.setOutputDataSet(transform.getOutput(), out); + } + } // -------------------------------------------------------------------------------------------- // Miscellaneous diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java index d524846a246f..5d63e2366d4c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java @@ -1,37 +1,42 @@ package com.dataartisans.flink.dataflow.translation.functions; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema; import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.TupleTag; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.util.Collector; +import java.util.ArrayList; +import java.util.List; + public class FlinkCoGroupKeyedListAggregator implements CoGroupFunction, KV, KV>{ -// private CoGbkResultSchema schema; -// private TupleTag tupleTag1; -// private TupleTag tupleTag2; + private CoGbkResultSchema schema; + private TupleTag tupleTag1; + private TupleTag tupleTag2; -// public CoGroupKeyedListAggregator(CoGbkResultSchema schema, TupleTag tupleTag1, TupleTag tupleTag2) { -// this.schema = schema; -// this.tupleTag1 = tupleTag1; -// this.tupleTag2 = tupleTag2; -// } + public FlinkCoGroupKeyedListAggregator(CoGbkResultSchema schema, TupleTag tupleTag1, TupleTag tupleTag2) { + this.schema = schema; + this.tupleTag1 = tupleTag1; + this.tupleTag2 = tupleTag2; + } @Override public void coGroup(Iterable> first, Iterable> second, Collector> out) throws Exception { -// K k = null; -// List result = new ArrayList<>(); -// int index1 = schema.getIndex(tupleTag1); -// for (KV entry : first) { -// k = entry.getKey(); -// result.add(new RawUnionValue(index1, entry.getValue())); -// } -// int index2 = schema.getIndex(tupleTag2); -// for (KV entry : second) { -// k = entry.getKey(); -// result.add(new RawUnionValue(index2, entry.getValue())); -// } -// out.collect(KV.of(k, new CoGbkResult(schema, result))); + K k = null; + List result = new ArrayList<>(); + int index1 = schema.getIndex(tupleTag1); + for (KV entry : first) { + k = entry.getKey(); + result.add(new RawUnionValue(index1, entry.getValue())); + } + int index2 = schema.getIndex(tupleTag2); + for (KV entry : second) { + k = entry.getKey(); + result.add(new RawUnionValue(index2, entry.getValue())); + } + out.collect(KV.of(k, new CoGbkResult(schema, (Iterable) result))); } } From ccbf40930fa0bd839f6a97956fdbb1cbf04ca82b Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 9 Mar 2015 18:52:15 +0100 Subject: [PATCH 053/149] integrate now available Google Dataflow classes UnionCoder is still not available => new pull request --- .../FlinkTransformTranslators.java | 15 ++---- .../FlinkCoGroupKeyedListAggregator.java | 4 +- .../FlinkMultiOutputDoFnFunction.java | 1 + .../FlinkMultiOutputPruningFunction.java | 1 + .../translation/functions/RawUnionValue.java | 50 ------------------- .../translation/functions/UnionCoder.java | 1 + 6 files changed, 9 insertions(+), 63 deletions(-) delete mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/RawUnionValue.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 7f3738564f0b..44b4498bf2ce 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema; import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; +import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionView; @@ -31,10 +32,8 @@ import com.google.common.collect.Lists; import org.apache.avro.Schema; import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.SortPartitionOperator; import org.apache.flink.api.java.io.AvroInputFormat; import org.apache.flink.api.java.io.AvroOutputFormat; import org.apache.flink.api.java.io.TextInputFormat; @@ -264,16 +263,8 @@ private static class CombinePerKeyTranslator implements FlinkPipe public void translateNode(Combine.PerKey transform, TranslationContext context) { DataSet> inputDataSet = context.getInputDataSet(transform.getInput()); - Combine.KeyedCombineFn keyedCombineFn = null; - // This is super hacky, but unfortunately we cannot get the fn otherwise - try { - Field fnField = transform.getClass().getDeclaredField("fn"); - fnField.setAccessible(true); - keyedCombineFn = (Combine.KeyedCombineFn) fnField.get(transform); - } catch (NoSuchFieldException | IllegalAccessException e) { - // we know that the field is there and it is accessible - System.out.println("Could not access KeyedCombineFn: " + e); - } + @SuppressWarnings("unchecked") + Combine.KeyedCombineFn keyedCombineFn = (Combine.KeyedCombineFn) transform.getFn(); KvCoder inputCoder = (KvCoder) transform.getInput().getCoder(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java index 5d63e2366d4c..dd17729d0911 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java @@ -2,6 +2,7 @@ import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema; +import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.TupleTag; import org.apache.flink.api.common.functions.CoGroupFunction; @@ -37,6 +38,7 @@ public void coGroup(Iterable> first, Iterable> second, Collect k = entry.getKey(); result.add(new RawUnionValue(index2, entry.getValue())); } - out.collect(KV.of(k, new CoGbkResult(schema, (Iterable) result))); + System.out.println("result: "+result); + out.collect(KV.of(k, new CoGbkResult(schema, (List) result))); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java index 17c8d9ead066..66f00ddd9e82 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -8,6 +8,7 @@ import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.util.WindowedValue; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java index 44a6daaa900c..a8133df7c846 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java @@ -17,6 +17,7 @@ */ package com.dataartisans.flink.dataflow.translation.functions; +import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/RawUnionValue.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/RawUnionValue.java deleted file mode 100644 index 0e8bf2678359..000000000000 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/RawUnionValue.java +++ /dev/null @@ -1,50 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dataartisans.flink.dataflow.translation.functions; - -/** - * This corresponds to an integer union tag and value. The mapping of - * union tag to type must come from elsewhere. - * - * This file copied from {@link com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue} - */ -public class RawUnionValue { - private final int unionTag; - private final Object value; - - /** - * Constructs a partial union from the given union tag and value. - */ - public RawUnionValue(int unionTag, Object value) { - this.unionTag = unionTag; - this.value = value; - } - - public int getUnionTag() { - return unionTag; - } - - public Object getValue() { - return value; - } - - @Override - public String toString() { - return unionTag + ":" + value; - } -} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java index 6480a9ff851f..99804473583d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.coders.StandardCoder; +import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.VarInt; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; From a7e66262f3de43f1c81638cfe5f1bedc7f427ae2 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 9 Mar 2015 19:01:28 +0100 Subject: [PATCH 054/149] integrate partialGroupReduce operator to properly optimize Combine.perKey --- .../FlinkTransformTranslators.java | 21 +++---- .../functions/FlinkPartialReduceFunction.java | 56 +++++++------------ .../functions/FlinkReduceFunction.java | 14 +++-- 3 files changed, 40 insertions(+), 51 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 44b4498bf2ce..22cfea894dba 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -272,27 +272,28 @@ public void translateNode(Combine.PerKey transform, TranslationContex keyedCombineFn.getAccumulatorCoder(transform.getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder()); + TypeInformation> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder); TypeInformation> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder)); - FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); + Grouping> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation)); + FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); - SortPartitionOperator> sortPartitionOperator = new SortPartitionOperator<> - (inputDataSet, "key", Order.ASCENDING, "Sort for PartialReduce: " + - transform.getName()); - - MapPartitionOperator, KV> partialReduceOperator = new - MapPartitionOperator<>(sortPartitionOperator, partialReduceTypeInfo, - partialReduceFunction, "PartialReduce: " + transform.getName()); + // Partially GroupReduce the values into the intermediate format VA + GroupReducePartialOperator, KV> partialGroupReduce = + new GroupReducePartialOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction, + "PartialGroupReduce: " + transform.getName()); GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); TypeInformation> reduceTypeInfo = context.getTypeInfo(transform.getOutput()); - Grouping> grouping = new UnsortedGrouping<>(partialReduceOperator, new Keys.ExpressionKeys<>(new String[]{"key"}, partialReduceOperator.getType())); + Grouping> intermediateGrouping = new UnsortedGrouping<>(partialGroupReduce, new Keys.ExpressionKeys<>(new String[]{"key"}, partialGroupReduce.getType())); + // Fully reduce the values and create output format VO GroupReduceOperator, KV> outputDataSet = - new GroupReduceOperator<>(grouping, reduceTypeInfo, reduceFunction, transform.getName()); + new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName()); + context.setOutputDataSet(transform.getOutput(), outputDataSet); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java index 0b1efb28a2b3..f55088b324d5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java @@ -3,56 +3,42 @@ import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.values.KV; import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.MapPartitionFunction; import org.apache.flink.util.Collector; import java.util.Iterator; /** - * Flink {@link org.apache.flink.api.common.functions.MapPartitionFunction} for executing a + * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input - * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and emits accumulated - * values. - * - * This assumes that the input values are sorted. We basically implement a GroupReduce operation - * inside a partition here. + * {@link com.google.cloud.dataflow.sdk.values.KV} elements VI, extracts the key and emits accumulated + * values which have the intermediate format VA. */ -public class FlinkPartialReduceFunction implements MapPartitionFunction, KV> { +public class FlinkPartialReduceFunction implements GroupReduceFunction, KV> { - private final Combine.KeyedCombineFn keyedCombineFn; + private final Combine.KeyedCombineFn keyedCombineFn; - public FlinkPartialReduceFunction(Combine.KeyedCombineFn + public FlinkPartialReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { this.keyedCombineFn = keyedCombineFn; } @Override - public void mapPartition(Iterable> values, Collector> out) throws - Exception { - Iterator> it = values.iterator(); - if (!it.hasNext()) { - return; + public void reduce(Iterable> elements, Collector> out) throws Exception { + + final Iterator> iterator = elements.iterator(); + // create accumulator using the first elements key + KV first = iterator.next(); + K key = first.getKey(); + VI value = first.getValue(); + VA accumulator = keyedCombineFn.createAccumulator(key); + // manually add for the first element + keyedCombineFn.addInput(key, accumulator, value); + + while(iterator.hasNext()) { + value = iterator.next().getValue(); + keyedCombineFn.addInput(key, accumulator, value); } - KV current = it.next(); - K currentKey = current.getKey(); - VA accumulator = keyedCombineFn.createAccumulator(currentKey); - keyedCombineFn.addInput(currentKey, accumulator, current.getValue()); - - while (it.hasNext()) { - current = it.next(); - if (currentKey == null && current.getKey() == null || current.getKey().equals(currentKey)) { - keyedCombineFn.addInput(currentKey, accumulator, current.getValue()); - } else { - // output current accumulation value, start new accumulation - out.collect(KV.of(currentKey, accumulator)); - currentKey = current.getKey(); - accumulator = keyedCombineFn.createAccumulator(currentKey); - keyedCombineFn.addInput(currentKey, accumulator, current.getValue()); - } - } - - // also emit last accumulation - out.collect(KV.of(currentKey, accumulator)); + out.collect(KV.of(key, accumulator)); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java index 9a12d1a8c4eb..e98bc564444b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java @@ -12,27 +12,29 @@ * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and merges the - * accumulators resulting from the PartialReduce in the MapPartition operation. + * accumulators resulting from the PartialReduce which produced the input VA. */ -public class FlinkReduceFunction implements GroupReduceFunction, KV> { +public class FlinkReduceFunction implements GroupReduceFunction, KV> { - private final Combine.KeyedCombineFn keyedCombineFn; + private final Combine.KeyedCombineFn keyedCombineFn; - public FlinkReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { + public FlinkReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { this.keyedCombineFn = keyedCombineFn; } @Override public void reduce(Iterable> values, Collector> out) throws Exception { Iterator> it = values.iterator(); + KV current = it.next(); K k = current.getKey(); VA accumulator = current.getValue(); + while (it.hasNext()) { current = it.next(); keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) ); } - out.collect(KV.of(k, keyedCombineFn.extractOutput(k, - accumulator))); + + out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator))); } } From fba7b27377703930367d01147e707bdc1969b2eb Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 9 Mar 2015 20:18:40 +0100 Subject: [PATCH 055/149] fix bug that would incorrectly match elements in the coGroup Ouch. --- .../flink/dataflow/translation/types/CoderComperator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java index 80bf41c64722..1afe007fdabf 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java @@ -66,7 +66,7 @@ public int compare(T first, T second) { byte[] arr = byteBuffer1.getBuffer(); byte[] arrOther = byteBuffer2.getBuffer(); if (byteBuffer1.size() != byteBuffer2.size()) { - return arr.length - arrOther.length; + return byteBuffer1.size() - byteBuffer2.size(); } int len = byteBuffer1.size(); for(int i = 0; i < len; i++ ) { From 2a6b19b651e51b27f87f86bb2836b9357cd31fe4 Mon Sep 17 00:00:00 2001 From: Max Date: Tue, 10 Mar 2015 15:47:21 +0100 Subject: [PATCH 056/149] implement a rudimentary normalized key comparator --- .../FlinkCoGroupKeyedListAggregator.java | 1 - .../translation/types/KvCoderComperator.java | 32 +++++++++++++++---- 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java index dd17729d0911..d974f80ae70e 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java @@ -38,7 +38,6 @@ public void coGroup(Iterable> first, Iterable> second, Collect k = entry.getKey(); result.add(new RawUnionValue(index2, entry.getValue())); } - System.out.println("result: "+result); out.collect(KV.of(k, new CoGbkResult(schema, (List) result))); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index 789b16d2c0e0..fd4e73d78847 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -9,6 +9,7 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.MemorySegment; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.ObjectInputStream; @@ -141,7 +142,7 @@ public int compareSerialized(DataInputView firstSource, DataInputView secondSour @Override public boolean supportsNormalizedKey() { - return false; + return true; } @Override @@ -151,27 +152,46 @@ public boolean supportsSerializationWithKeyNormalization() { @Override public int getNormalizeKeyLen() { - return 0; + return 8; } @Override public boolean isNormalizedKeyPrefixOnly(int keyBytes) { - return false; + return true; } @Override - public void putNormalizedKey(KV record, MemorySegment target, int offset, int numBytes) { + public void putNormalizedKey(KV record, MemorySegment target, int offset, int len) { + final int limit = offset + len; + + Coder keyCoder = coder.getKeyCoder(); + final int maxBytesPrefix = 8; + + ByteArrayOutputStream baos = new ByteArrayOutputStream(maxBytesPrefix); + try { + keyCoder.encode(record.getKey(), baos, Coder.Context.NESTED); + byte[] arr = baos.toByteArray(); + for(int i = 0; i < arr.length && i < maxBytesPrefix && offset < limit; i++) { + target.put(offset++, arr[i]); + } + } catch (IOException e) { + e.printStackTrace(); + } + + while (offset < limit) { + target.put(offset++, (byte) 0); + } } @Override public void writeWithKeyNormalization(KV record, DataOutputView target) throws IOException { - + throw new UnsupportedOperationException(); } @Override public KV readWithKeyDenormalization(KV reuse, DataInputView source) throws IOException { - return null; + throw new UnsupportedOperationException(); } @Override From 8820c095fe407057d58534cce7ad6af4e5d8a274 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 2 Mar 2015 13:41:00 +0100 Subject: [PATCH 057/149] Optimize CoderSerializer and CoderComparator We now keep the reference encoded in a byte array and reuse buffers and stuff between calls. --- .../translation/types/CoderComperator.java | 105 +++++++++---- .../types/CoderTypeSerializer.java | 15 +- .../translation/types/KvCoderComperator.java | 139 +++++++++++------- 3 files changed, 169 insertions(+), 90 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java index 1afe007fdabf..4d1a3c10bc29 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java @@ -15,25 +15,30 @@ */ public class CoderComperator extends TypeComparator { - private T reference = null; private Coder coder; // We use these for internal encoding/decoding for creating copies and comparing // serialized forms using a Coder - private transient InspectableByteArrayOutputStream byteBuffer1; - private transient InspectableByteArrayOutputStream byteBuffer2; + private transient InspectableByteArrayOutputStream buffer1; + private transient InspectableByteArrayOutputStream buffer2; + + // For storing the Reference in encoded form + private transient InspectableByteArrayOutputStream referenceBuffer; public CoderComperator(Coder coder) { this.coder = coder; - byteBuffer1 = new InspectableByteArrayOutputStream(); - byteBuffer2 = new InspectableByteArrayOutputStream(); + buffer1 = new InspectableByteArrayOutputStream(); + buffer2 = new InspectableByteArrayOutputStream(); + referenceBuffer = new InspectableByteArrayOutputStream(); } private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - byteBuffer1 = new InspectableByteArrayOutputStream(); - byteBuffer2 = new InspectableByteArrayOutputStream(); + buffer1 = new InspectableByteArrayOutputStream(); + buffer2 = new InspectableByteArrayOutputStream(); + referenceBuffer = new InspectableByteArrayOutputStream(); + } @Override @@ -43,32 +48,67 @@ public int hash(T record) { @Override public void setReference(T toCompare) { - this.reference = toCompare; + referenceBuffer.reset(); + try { + coder.encode(toCompare, referenceBuffer, Coder.Context.OUTER); + } catch (IOException e) { + throw new RuntimeException("Could not set reference " + toCompare + ": " + e); + } } @Override public boolean equalToReference(T candidate) { - return reference.equals(candidate); + try { + buffer2.reset(); + coder.encode(candidate, buffer2, Coder.Context.OUTER); + byte[] arr = referenceBuffer.getBuffer(); + byte[] arrOther = buffer2.getBuffer(); + if (referenceBuffer.size() != buffer2.size()) { + return false; + } + int len = buffer2.size(); + for(int i = 0; i < len; i++ ) { + if (arr[i] != arrOther[i]) { + return false; + } + } + return true; + } catch (IOException e) { + throw new RuntimeException("Could not compare reference.", e); + } } @Override public int compareToReference(TypeComparator other) { - return compare(this.reference, ((CoderComperator) other).reference); + InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComperator) other).referenceBuffer; + + byte[] arr = referenceBuffer.getBuffer(); + byte[] arrOther = otherReferenceBuffer.getBuffer(); + if (referenceBuffer.size() != otherReferenceBuffer.size()) { + return referenceBuffer.size() - otherReferenceBuffer.size(); + } + int len = referenceBuffer.size(); + for (int i = 0; i < len; i++) { + if (arr[i] != arrOther[i]) { + return arr[i] - arrOther[i]; + } + } + return 0; } @Override public int compare(T first, T second) { try { - byteBuffer1.reset(); - byteBuffer2.reset(); - coder.encode(first, byteBuffer1, Coder.Context.OUTER); - coder.encode(second, byteBuffer2, Coder.Context.OUTER); - byte[] arr = byteBuffer1.getBuffer(); - byte[] arrOther = byteBuffer2.getBuffer(); - if (byteBuffer1.size() != byteBuffer2.size()) { - return byteBuffer1.size() - byteBuffer2.size(); + buffer1.reset(); + buffer2.reset(); + coder.encode(first, buffer1, Coder.Context.OUTER); + coder.encode(second, buffer2, Coder.Context.OUTER); + byte[] arr = buffer1.getBuffer(); + byte[] arrOther = buffer2.getBuffer(); + if (buffer1.size() != buffer2.size()) { + return buffer1.size() - buffer2.size(); } - int len = byteBuffer1.size(); + int len = buffer1.size(); for(int i = 0; i < len; i++ ) { if (arr[i] != arrOther[i]) { return arr[i] - arrOther[i]; @@ -76,7 +116,7 @@ public int compare(T first, T second) { } return 0; } catch (IOException e) { - throw new RuntimeException("Could not compare reference.", e); + throw new RuntimeException("Could not compare: ", e); } } @@ -90,7 +130,7 @@ public int compareSerialized(DataInputView firstSource, DataInputView secondSour @Override public boolean supportsNormalizedKey() { - return false; + return true; } @Override @@ -100,27 +140,42 @@ public boolean supportsSerializationWithKeyNormalization() { @Override public int getNormalizeKeyLen() { - return 0; + return Integer.MAX_VALUE; } @Override public boolean isNormalizedKeyPrefixOnly(int keyBytes) { - return false; + return true; } @Override public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) { + buffer1.reset(); + try { + coder.encode(record, buffer1, Coder.Context.OUTER); + } catch (IOException e) { + throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e); + } + final byte[] data = buffer1.getBuffer(); + final int limit = offset + numBytes; + + target.put(offset, data, 0, buffer1.size()); + offset += buffer1.size(); + + while (offset < limit) { + target.put(offset++, (byte) 0); + } } @Override public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException { - + throw new UnsupportedOperationException(); } @Override public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException { - return null; + throw new UnsupportedOperationException(); } @Override diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java index d3b980157ad2..aea1727b39c0 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java @@ -24,14 +24,14 @@ public class CoderTypeSerializer extends TypeSerializer { private transient DataOutputViewWrapper outputWrapper; // We use this for internal encoding/decoding for creating copies using the Coder. - private transient InspectableByteArrayOutputStream byteBuffer; + private transient InspectableByteArrayOutputStream buffer; public CoderTypeSerializer(Coder coder) { this.coder = coder; this.inputWrapper = new DataInputViewWrapper(null); this.outputWrapper = new DataOutputViewWrapper(null); - byteBuffer = new InspectableByteArrayOutputStream(); + buffer = new InspectableByteArrayOutputStream(); } private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { @@ -39,7 +39,7 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE this.inputWrapper = new DataInputViewWrapper(null); this.outputWrapper = new DataOutputViewWrapper(null); - byteBuffer = new InspectableByteArrayOutputStream(); + buffer = new InspectableByteArrayOutputStream(); } @Override @@ -59,16 +59,15 @@ public T createInstance() { @Override public T copy(T t) { - byteBuffer.reset(); + buffer.reset(); try { - coder.encode(t, byteBuffer, Coder.Context.OUTER); + coder.encode(t, buffer, Coder.Context.OUTER); } catch (IOException e) { throw new RuntimeException("Could not copy.", e); } try { - return coder.decode( - new ByteArrayInputStream(byteBuffer.getBuffer(), 0, byteBuffer.size()), - Coder.Context.OUTER); + return coder.decode(new ByteArrayInputStream(buffer.getBuffer(), 0, buffer + .size()), Coder.Context.OUTER); } catch (IOException e) { throw new RuntimeException("Could not copy.", e); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index fd4e73d78847..b9ec8d08314b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -20,22 +20,28 @@ */ public class KvCoderComperator extends TypeComparator> { - private KV reference = null; private KvCoder coder; + private Coder keyCoder; // We use these for internal encoding/decoding for creating copies and comparing // serialized forms using a Coder - private transient InspectableByteArrayOutputStream byteBuffer1; - private transient InspectableByteArrayOutputStream byteBuffer2; + private transient InspectableByteArrayOutputStream buffer1; + private transient InspectableByteArrayOutputStream buffer2; + + // For storing the Reference in encoded form + private transient InspectableByteArrayOutputStream referenceBuffer; + // For deserializing the key private transient DataInputViewWrapper inputWrapper; public KvCoderComperator(KvCoder coder) { this.coder = coder; + this.keyCoder = coder.getKeyCoder(); - byteBuffer1 = new InspectableByteArrayOutputStream(); - byteBuffer2 = new InspectableByteArrayOutputStream(); + buffer1 = new InspectableByteArrayOutputStream(); + buffer2 = new InspectableByteArrayOutputStream(); + referenceBuffer = new InspectableByteArrayOutputStream(); inputWrapper = new DataInputViewWrapper(null); } @@ -43,16 +49,13 @@ public KvCoderComperator(KvCoder coder) { private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - byteBuffer1 = new InspectableByteArrayOutputStream(); - byteBuffer2 = new InspectableByteArrayOutputStream(); + buffer1 = new InspectableByteArrayOutputStream(); + buffer2 = new InspectableByteArrayOutputStream(); + referenceBuffer = new InspectableByteArrayOutputStream(); inputWrapper = new DataInputViewWrapper(null); } - public KV getReference() { - return reference; - } - @Override public int hash(KV record) { K key = record.getKey(); @@ -65,40 +68,66 @@ public int hash(KV record) { @Override public void setReference(KV toCompare) { - this.reference = toCompare; + referenceBuffer.reset(); + try { + keyCoder.encode(toCompare.getKey(), referenceBuffer, Coder.Context.OUTER); + } catch (IOException e) { + throw new RuntimeException("Could not set reference " + toCompare + ": " + e); + } } @Override public boolean equalToReference(KV candidate) { - K key = reference.getKey(); - K otherKey = candidate.getKey(); - if (key == null && otherKey == null) { + try { + buffer2.reset(); + keyCoder.encode(candidate.getKey(), buffer2, Coder.Context.OUTER); + byte[] arr = referenceBuffer.getBuffer(); + byte[] arrOther = buffer2.getBuffer(); + if (referenceBuffer.size() != buffer2.size()) { + return false; + } + int len = buffer2.size(); + for(int i = 0; i < len; i++ ) { + if (arr[i] != arrOther[i]) { + return false; + } + } return true; - } else if(key == null || otherKey == null) { - return false; - } else { - return key.equals(otherKey); + } catch (IOException e) { + throw new RuntimeException("Could not compare reference.", e); } } @Override public int compareToReference(TypeComparator> other) { - return compare(this.reference, ((KvCoderComperator) other).reference); - } + InspectableByteArrayOutputStream otherReferenceBuffer = ((KvCoderComperator) other).referenceBuffer; + + byte[] arr = referenceBuffer.getBuffer(); + byte[] arrOther = otherReferenceBuffer.getBuffer(); + if (referenceBuffer.size() != otherReferenceBuffer.size()) { + return referenceBuffer.size() - otherReferenceBuffer.size(); + } + int len = referenceBuffer.size(); + for (int i = 0; i < len; i++) { + if (arr[i] != arrOther[i]) { + return arr[i] - arrOther[i]; + } + } + return 0; } @Override public int compare(KV first, KV second) { try { - byteBuffer1.reset(); - byteBuffer2.reset(); - coder.getKeyCoder().encode(first.getKey(), byteBuffer1, Coder.Context.OUTER); - coder.getKeyCoder().encode(second.getKey(), byteBuffer2, Coder.Context.OUTER); - byte[] arr = byteBuffer1.getBuffer(); - byte[] arrOther = byteBuffer2.getBuffer(); - if (byteBuffer1.size() != byteBuffer2.size()) { - return byteBuffer1.size() - byteBuffer2.size(); + buffer1.reset(); + buffer2.reset(); + keyCoder.encode(first.getKey(), buffer1, Coder.Context.OUTER); + keyCoder.encode(second.getKey(), buffer2, Coder.Context.OUTER); + byte[] arr = buffer1.getBuffer(); + byte[] arrOther = buffer2.getBuffer(); + if (buffer1.size() != buffer2.size()) { + return buffer1.size() - buffer2.size(); } - int len = byteBuffer1.size(); + int len = buffer1.size(); for(int i = 0; i < len; i++ ) { if (arr[i] != arrOther[i]) { return arr[i] - arrOther[i]; @@ -114,21 +143,21 @@ public int compare(KV first, KV second) { public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { CoderTypeSerializer> serializer = new CoderTypeSerializer>(coder); inputWrapper.setInputView(firstSource); - K firstKey = coder.getKeyCoder().decode(inputWrapper, Coder.Context.NESTED); + K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); inputWrapper.setInputView(secondSource); - K secondKey = coder.getKeyCoder().decode(inputWrapper, Coder.Context.NESTED); + K secondKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); try { - byteBuffer1.reset(); - byteBuffer2.reset(); - coder.getKeyCoder().encode(firstKey, byteBuffer1, Coder.Context.OUTER); - coder.getKeyCoder().encode(secondKey, byteBuffer2, Coder.Context.OUTER); - byte[] arr = byteBuffer1.getBuffer(); - byte[] arrOther = byteBuffer2.getBuffer(); - if (byteBuffer1.size() != byteBuffer2.size()) { - return byteBuffer1.size() - byteBuffer2.size(); + buffer1.reset(); + buffer2.reset(); + keyCoder.encode(firstKey, buffer1, Coder.Context.OUTER); + keyCoder.encode(secondKey, buffer2, Coder.Context.OUTER); + byte[] arr = buffer1.getBuffer(); + byte[] arrOther = buffer2.getBuffer(); + if (buffer1.size() != buffer2.size()) { + return buffer1.size() - buffer2.size(); } - int len = byteBuffer1.size(); + int len = buffer1.size(); for(int i = 0; i < len; i++ ) { if (arr[i] != arrOther[i]) { return arr[i] - arrOther[i]; @@ -152,7 +181,7 @@ public boolean supportsSerializationWithKeyNormalization() { @Override public int getNormalizeKeyLen() { - return 8; + return Integer.MAX_VALUE; } @Override @@ -161,23 +190,19 @@ public boolean isNormalizedKeyPrefixOnly(int keyBytes) { } @Override - public void putNormalizedKey(KV record, MemorySegment target, int offset, int len) { - final int limit = offset + len; - - Coder keyCoder = coder.getKeyCoder(); - - final int maxBytesPrefix = 8; - - ByteArrayOutputStream baos = new ByteArrayOutputStream(maxBytesPrefix); + public void putNormalizedKey(KV record, MemorySegment target, int offset, int numBytes) { + buffer1.reset(); try { - keyCoder.encode(record.getKey(), baos, Coder.Context.NESTED); - byte[] arr = baos.toByteArray(); - for(int i = 0; i < arr.length && i < maxBytesPrefix && offset < limit; i++) { - target.put(offset++, arr[i]); - } + keyCoder.encode(record.getKey(), buffer1, Coder.Context.OUTER); } catch (IOException e) { - e.printStackTrace(); + throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e); } + final byte[] data = buffer1.getBuffer(); + final int limit = offset + numBytes; + + target.put(offset, data, 0, buffer1.size()); + + offset += buffer1.size(); while (offset < limit) { target.put(offset++, (byte) 0); @@ -214,6 +239,6 @@ public int extractKeys(Object record, Object[] target, int index) { @Override public TypeComparator[] getFlatComparators() { - return new TypeComparator[] {new CoderComperator<>(coder.getKeyCoder())}; + return new TypeComparator[] {new CoderComperator<>(keyCoder)}; } } From 555f06f37a147fb975e8403b424c88fd5915695f Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 11 Mar 2015 14:07:39 +0100 Subject: [PATCH 058/149] fix buffer overflow --- .../flink/dataflow/translation/types/CoderComperator.java | 2 +- .../flink/dataflow/translation/types/KvCoderComperator.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java index 4d1a3c10bc29..6f1288d7e018 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java @@ -159,7 +159,7 @@ public void putNormalizedKey(T record, MemorySegment target, int offset, int num final byte[] data = buffer1.getBuffer(); final int limit = offset + numBytes; - target.put(offset, data, 0, buffer1.size()); + target.put(offset, data, 0, Math.min(numBytes, buffer1.size())); offset += buffer1.size(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index b9ec8d08314b..14c1b837ef8c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -200,7 +200,7 @@ public void putNormalizedKey(KV record, MemorySegment target, int offset, final byte[] data = buffer1.getBuffer(); final int limit = offset + numBytes; - target.put(offset, data, 0, buffer1.size()); + target.put(offset, data, 0, Math.min(numBytes, buffer1.size())); offset += buffer1.size(); From e0481e0d0ac4e8ef2755ffb4171d93f695972d8c Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 11 Mar 2015 20:04:08 +0100 Subject: [PATCH 059/149] add README file --- runners/flink/README.md | 70 +++++++++++++++++++++++++++++++++++++++-- 1 file changed, 68 insertions(+), 2 deletions(-) diff --git a/runners/flink/README.md b/runners/flink/README.md index c6cf8487f985..488b3f8c86cf 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -1,2 +1,68 @@ -# flink-dataflow -Google Dataflow Runner for Apache Flink +Flink-Dataflow +-------------- + +Flink-Dataflow is a Google Dataflow Runner for Apache Flink. It enables you to +run Dataflow programs with Flink as an execution engine. + +# Getting Started + +To get started using Google Dataflow on top of Apache Flink, we need to install the +latest version of Flink and Flink-Dataflow. + +## Install Flink ## + +To install the latest version of Flink, execute the following: + + git clone https://github.com/apache/flink.git + +Then build and install Flink in your local Maven repository. This will take some minutes. + + mvn clean install -DskipTests + +## Install Flink-Dataflow ## + +To retrieve the latest version of Flink-Dataflow, run the following command + + git clone https://github.com/dataArtisans/flink-dataflow + +Then switch to the newly created directory and run Maven to build the Dataflow runner: + + cd flink-dataflow + mvn clean install -DskipTests + +Flink-Dataflow is now installed in your local maven repository. + +## Executing an example + +Next, let's run the classic WordCount example. It's semantically identically to +the example provided with Google Dataflow. Only this time, we chose the +`FlinkPipelineRunner` to execute the WordCount on top of Flink. + +Here's an excerpt from the WordCount class file: + +```java +Options options = PipelineOptionsFactory.fromArgs(args).as(Options.class); +// yes, we want to run WordCount with Flink +options.setRunner(FlinkPipelineRunner.class); + +Pipeline p = Pipeline.create(options); + +p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(new CountWords()) + .apply(TextIO.Write.named("WriteCounts") + .to(options.getOutput()) + .withNumShards(options.getNumShards())); + +p.run(); +``` + + +To execute the example, get some sample data and run: + + + curl http://www.gutenberg.org/cache/epub/1128/pg1128.txt > kinglear.txt + mvn exec:java -Dexec.mainClass=com.dataartisans.flink.dataflow.examples.WordCount \ + -Dexec.args="--input=kinglear.txt --output=wordcounts.txt" + + +Congratulations, you have run your first Google Dataflow program on top of Apache Flink! From 254cf2a698fec503591688bec5a187d6fb2e4e97 Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 11 Mar 2015 20:31:16 +0100 Subject: [PATCH 060/149] add license headers --- .../flink/dataflow/FlinkPipelineOptions.java | 16 +++++++--------- .../flink/dataflow/FlinkPipelineRunner.java | 15 +++++++++++++++ .../flink/dataflow/FlinkRunnerResult.java | 16 +++++++++++++++- .../flink/dataflow/examples/AnotherJOb.java | 18 +++++++++++++++--- .../dataflow/examples/FlattenizeThis.java | 15 +++++++++++++++ .../dataflow/examples/MaybeEmptyTest.java | 15 +++++++++++++++ .../dataflow/examples/ParDoMultiOutput.java | 15 +++++++++++++++ .../flink/dataflow/examples/SideInputTest.java | 15 +++++++++++++++ .../flink/dataflow/examples/TFIDF.java | 15 +++++++++++++++ .../flink/dataflow/examples/WordCount.java | 15 +++++++++++++++ .../flink/dataflow/io/ConsoleIO.java | 16 +++++++--------- .../translation/FlinkPipelineTranslator.java | 17 +++++++++++++++++ .../translation/FlinkTransformTranslators.java | 17 +++++++++++++++++ .../translation/TranslationContext.java | 17 +++++++++++++++++ .../FlinkCoGroupKeyedListAggregator.java | 15 +++++++++++++++ .../functions/FlinkCreateFunction.java | 15 +++++++++++++++ .../functions/FlinkDoFnFunction.java | 15 +++++++++++++++ .../FlinkKeyedListAggregationFunction.java | 15 +++++++++++++++ .../FlinkMultiOutputDoFnFunction.java | 15 +++++++++++++++ .../FlinkMultiOutputPruningFunction.java | 16 +++++++--------- .../functions/FlinkPartialReduceFunction.java | 17 +++++++++++++++++ .../functions/FlinkReduceFunction.java | 17 +++++++++++++++++ .../translation/functions/UnionCoder.java | 18 +++++++++--------- .../translation/types/CoderComperator.java | 17 +++++++++++++++++ .../types/CoderTypeInformation.java | 17 +++++++++++++++++ .../translation/types/CoderTypeSerializer.java | 17 +++++++++++++++++ .../InspectableByteArrayOutputStream.java | 18 +++++++++--------- .../translation/types/KvCoderComperator.java | 17 +++++++++++++++++ .../types/KvCoderTypeInformation.java | 17 +++++++++++++++++ .../types/VoidCoderTypeSerializer.java | 17 +++++++++++++++++ .../wrappers/CombineFnAggregatorWrapper.java | 17 +++++++++++++++++ .../wrappers/DataInputViewWrapper.java | 17 +++++++++++++++++ .../wrappers/DataOutputViewWrapper.java | 17 +++++++++++++++++ .../SerializableFnAggregatorWrapper.java | 17 +++++++++++++++++ .../wrappers/SourceInputFormat.java | 18 +++++++++--------- .../translation/wrappers/SourceInputSplit.java | 18 +++++++++--------- .../flink/dataflow/AvroITCase.java | 16 +++++++--------- .../flink/dataflow/FlinkTestPipeline.java | 16 +++++++--------- .../flink/dataflow/JoinExamples.java | 16 +++++++--------- .../flink/dataflow/JoinExamplesITCase.java | 16 +++++++--------- .../flink/dataflow/ReadSourceITCase.java | 16 +++++++--------- .../dataflow/RemoveDuplicatesEmptyITCase.java | 16 +++++++--------- .../flink/dataflow/RemoveDuplicatesITCase.java | 16 +++++++--------- .../flink/dataflow/TfIdfITCase.java | 16 +++++++--------- .../flink/dataflow/TopWikipediaSessions.java | 16 +++++++--------- .../dataflow/TopWikipediaSessionsITCase.java | 16 +++++++--------- .../flink/dataflow/WordCountITCase.java | 16 +++++++--------- .../flink/dataflow/WordCountJoin2ITCase.java | 15 +++++++++++++++ .../flink/dataflow/WordCountJoin3ITCase.java | 15 +++++++++++++++ 49 files changed, 629 insertions(+), 166 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java index 610502cd0de4..7a7f2402598e 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * 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, diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java index 0966d0a6d7cd..ed553ca87998 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow; import com.dataartisans.flink.dataflow.translation.FlinkPipelineTranslator; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java index a4adac01f30b..56cb297da8a9 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java @@ -1,7 +1,21 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow; import com.google.cloud.dataflow.sdk.PipelineResult; - import java.util.Collections; import java.util.Map; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/AnotherJOb.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/AnotherJOb.java index f124e166fb99..7842605cf6af 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/AnotherJOb.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/AnotherJOb.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.examples; import com.dataartisans.flink.dataflow.FlinkPipelineRunner; @@ -8,9 +23,6 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; -/** - * Created by max on 16/02/15. - */ public class AnotherJOb { public static void main(String[] args) { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/FlattenizeThis.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/FlattenizeThis.java index ccf576dd82dc..fafff4d4e894 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/FlattenizeThis.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/FlattenizeThis.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.examples; import com.dataartisans.flink.dataflow.FlinkPipelineRunner; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/MaybeEmptyTest.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/MaybeEmptyTest.java index 5a89334d1ff7..24257c8d1471 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/MaybeEmptyTest.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/MaybeEmptyTest.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.examples; import com.dataartisans.flink.dataflow.FlinkPipelineRunner; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/ParDoMultiOutput.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/ParDoMultiOutput.java index f69184d84564..5df3d86cff9f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/ParDoMultiOutput.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/ParDoMultiOutput.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.examples; import com.dataartisans.flink.dataflow.FlinkPipelineRunner; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java index c18e2a3fad9f..e83347a3e044 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.examples; import com.dataartisans.flink.dataflow.FlinkPipelineRunner; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java index d6bf1d35863e..e425c1688424 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.examples; import com.dataartisans.flink.dataflow.FlinkPipelineRunner; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java index 4981350e3ef4..9521d5579c89 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.examples; import com.dataartisans.flink.dataflow.FlinkPipelineOptions; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java index baef872757f4..aeac1ea95eec 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * 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, diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java index b4b90c75743b..8d2ece2918f5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation; import com.google.cloud.dataflow.sdk.Pipeline; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 22cfea894dba..6d975513672f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation; import com.dataartisans.flink.dataflow.io.ConsoleIO; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index 400c97a81b97..887d94016d85 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation; import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java index d974f80ae70e..f8593488100f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.functions; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java index bc1ff90cd9db..0e6945f0fb9b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.functions; import com.google.cloud.dataflow.sdk.coders.Coder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java index 6c22f7a68f10..e0c771a293f0 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.functions; import com.dataartisans.flink.dataflow.translation.wrappers.CombineFnAggregatorWrapper; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java index 8b5766557c05..48da66cb7d9d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.functions; import com.google.cloud.dataflow.sdk.values.KV; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java index 66f00ddd9e82..95bdadf0776d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.functions; import com.dataartisans.flink.dataflow.translation.wrappers.CombineFnAggregatorWrapper; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java index a8133df7c846..6792b23ec93b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * 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, diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java index f55088b324d5..92326c14ce31 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.functions; import com.google.cloud.dataflow.sdk.transforms.Combine; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java index e98bc564444b..cd0b38c183bb 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.functions; import com.google.cloud.dataflow.sdk.transforms.Combine; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java index 99804473583d..a246a52c4743 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java @@ -1,13 +1,13 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * * - * 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, diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java index 6f1288d7e018..38569824dbfb 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.types; import com.google.cloud.dataflow.sdk.coders.Coder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java index 3a0fdd5f7619..e249a6cb73b2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.types; import com.google.cloud.dataflow.sdk.coders.Coder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java index aea1727b39c0..58357cd9d0c6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.types; import com.dataartisans.flink.dataflow.translation.wrappers.DataInputViewWrapper; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java index 74c963bc482e..5d918cc7e9d1 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java @@ -1,13 +1,13 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * * - * 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, diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index 14c1b837ef8c..5d9f348a15d0 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.types; import com.dataartisans.flink.dataflow.translation.wrappers.DataInputViewWrapper; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java index b77f7ebc29db..5935bf9c481b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.types; import com.google.cloud.dataflow.sdk.coders.Coder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java index f4517d22f304..abad14b1721e 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.types; import org.apache.flink.api.common.typeutils.TypeSerializer; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java index 6cde53cc47e6..16e2bdcc7c04 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers; import com.google.cloud.dataflow.sdk.transforms.Aggregator; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java index d311bf7f169a..90582b06693d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers; import org.apache.flink.core.memory.DataInputView; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java index d3b930382f61..46df8e52defa 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers; import org.apache.flink.core.memory.DataOutputView; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java index 05640f6b30cf..acf9a4d98451 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -1,3 +1,20 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers; import com.google.cloud.dataflow.sdk.transforms.Aggregator; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java index 87b511dfc093..11d4c75bac8d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java @@ -1,13 +1,13 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * * - * 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, diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java index 45770041f5d8..0c5165e9531c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java @@ -1,13 +1,13 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * * - * 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java index fae0974baee4..e9efff239a0b 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH * - * http://www.apache.org/licenses/LICENSE-2.0 + * Licensed 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java index 61cb2d6f1882..a1f66c79f205 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamples.java index 1413bce66d36..648e54804648 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamples.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamples.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java index 3785dafaa771..3e8109423312 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java index 78ece6e3ae49..08a63b6b911b 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH * - * http://www.apache.org/licenses/LICENSE-2.0 + * Licensed 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java index 867474d35ec8..d4f5ae2bd8b1 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH * - * http://www.apache.org/licenses/LICENSE-2.0 + * Licensed 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java index 3e8d075dcd3c..a8200aa4c12d 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH * - * http://www.apache.org/licenses/LICENSE-2.0 + * Licensed 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java index f441f4973765..457c70366e30 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH * - * http://www.apache.org/licenses/LICENSE-2.0 + * Licensed 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java index c90d825223b2..07b6adfdeb5b 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 + * 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java index 751d5c827d89..9c8147bd4a48 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java @@ -1,13 +1,11 @@ -///** -// * 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 +///* +// * Copyright 2015 Data Artisans GmbH // * -// * http://www.apache.org/licenses/LICENSE-2.0 +// * Licensed 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java index 7501f1dedb6a..7bfa0ba8def6 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java @@ -1,13 +1,11 @@ -/** - * 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 +/* + * Copyright 2015 Data Artisans GmbH * - * http://www.apache.org/licenses/LICENSE-2.0 + * Licensed 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, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java index 6742adcec42a..caddc289ff1c 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow; import com.google.cloud.dataflow.examples.WordCount; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java index f0b78628edd3..585b84c98127 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java @@ -1,3 +1,18 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow; import com.google.cloud.dataflow.examples.WordCount; From caa2baa482faee201c293d07d4cc5fbd3c4730c2 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 13 Mar 2015 18:15:33 +0100 Subject: [PATCH 061/149] change name of Flink's new GroupCombine operator --- .../dataflow/translation/FlinkTransformTranslators.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 6d975513672f..3b92a41fc0a3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -297,15 +297,15 @@ public void translateNode(Combine.PerKey transform, TranslationContex FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); // Partially GroupReduce the values into the intermediate format VA - GroupReducePartialOperator, KV> partialGroupReduce = - new GroupReducePartialOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction, - "PartialGroupReduce: " + transform.getName()); + GroupCombineOperator, KV> groupCombine = + new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction, + "GroupCombine: " + transform.getName()); GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); TypeInformation> reduceTypeInfo = context.getTypeInfo(transform.getOutput()); - Grouping> intermediateGrouping = new UnsortedGrouping<>(partialGroupReduce, new Keys.ExpressionKeys<>(new String[]{"key"}, partialGroupReduce.getType())); + Grouping> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); // Fully reduce the values and create output format VO GroupReduceOperator, KV> outputDataSet = From 6af613d7af79ddf9391131ed027370d8f360e18f Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 23 Mar 2015 12:37:10 +0100 Subject: [PATCH 062/149] properly integrate Flink's new GroupCombine operator --- .../dataflow/translation/FlinkTransformTranslators.java | 3 ++- .../translation/functions/FlinkPartialReduceFunction.java | 7 ++++--- .../dataflow/translation/types/KvCoderComperator.java | 1 + 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 3b92a41fc0a3..09bafef58e96 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -296,11 +296,12 @@ public void translateNode(Combine.PerKey transform, TranslationContex FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); - // Partially GroupReduce the values into the intermediate format VA + // Partially GroupReduce the values into the intermediate format VA (combine) GroupCombineOperator, KV> groupCombine = new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction, "GroupCombine: " + transform.getName()); + // Reduce fully to VO GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); TypeInformation> reduceTypeInfo = context.getTypeInfo(transform.getOutput()); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java index 92326c14ce31..ecb897f32872 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java @@ -19,18 +19,19 @@ import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.values.KV; +import org.apache.flink.api.common.functions.FlatCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.util.Collector; import java.util.Iterator; /** - * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a + * Flink {@link org.apache.flink.api.common.functions.FlatCombineFunction} for executing a * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input * {@link com.google.cloud.dataflow.sdk.values.KV} elements VI, extracts the key and emits accumulated * values which have the intermediate format VA. */ -public class FlinkPartialReduceFunction implements GroupReduceFunction, KV> { +public class FlinkPartialReduceFunction implements FlatCombineFunction, KV> { private final Combine.KeyedCombineFn keyedCombineFn; @@ -40,7 +41,7 @@ public FlinkPartialReduceFunction(Combine.KeyedCombineFn } @Override - public void reduce(Iterable> elements, Collector> out) throws Exception { + public void combine(Iterable> elements, Collector> out) throws Exception { final Iterator> iterator = elements.iterator(); // create accumulator using the first elements key diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index 5d9f348a15d0..30868fb53cc5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -132,6 +132,7 @@ public int compareToReference(TypeComparator> other) { } return 0; } + @Override public int compare(KV first, KV second) { try { From d6a728d2cfec3a7c6c795a3bbd11709ce0abe88d Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 23 Mar 2015 12:53:04 +0100 Subject: [PATCH 063/149] README: remove install Flink part due to availability of maven dependency --- runners/flink/README.md | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/runners/flink/README.md b/runners/flink/README.md index 488b3f8c86cf..aebfd1706bdd 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -7,17 +7,7 @@ run Dataflow programs with Flink as an execution engine. # Getting Started To get started using Google Dataflow on top of Apache Flink, we need to install the -latest version of Flink and Flink-Dataflow. - -## Install Flink ## - -To install the latest version of Flink, execute the following: - - git clone https://github.com/apache/flink.git - -Then build and install Flink in your local Maven repository. This will take some minutes. - - mvn clean install -DskipTests +latest version of Flink-Dataflow. ## Install Flink-Dataflow ## From 40f0222300c4ffe4764c62bd9cda9f24c89888d7 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 23 Mar 2015 15:23:42 +0100 Subject: [PATCH 064/149] fix null value issue in Create translator --- .../translation/FlinkTransformTranslators.java | 2 +- .../translation/functions/FlinkCreateFunction.java | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 09bafef58e96..98958c6d68bd 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -449,7 +449,7 @@ public void translateNode(Create transform, TranslationContext context) { TypeInformation typeInformation = context.getTypeInfo(transform.getOutput()); Iterable elements = transform.getElements(); - // we need to serializer the elements to byte arrays, since they might contain + // we need to serialize the elements to byte arrays, since they might contain // elements that are not serializable by Java serialization. We deserialize them // in the FlatMap function using the Coder. diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java index 0e6945f0fb9b..c5b9aa67a4f5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java @@ -15,6 +15,7 @@ */ package com.dataartisans.flink.dataflow.translation.functions; +import com.dataartisans.flink.dataflow.translation.types.VoidCoderTypeSerializer; import com.google.cloud.dataflow.sdk.coders.Coder; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; @@ -40,10 +41,18 @@ public FlinkCreateFunction(List elements, Coder coder) { } @Override + @SuppressWarnings("unchecked") public void flatMap(IN value, Collector out) throws Exception { + for (byte[] element : elements) { ByteArrayInputStream bai = new ByteArrayInputStream(element); - out.collect(coder.decode(bai, Coder.Context.OUTER)); + OUT outValue = coder.decode(bai, Coder.Context.OUTER); + if (outValue == null) { + // TODO Flink doesn't allow null values in records + out.collect((OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE); + } else { + out.collect(outValue); + } } } } From d470f961c39b593175261206f09a6d661137deb9 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 23 Mar 2015 16:31:44 +0100 Subject: [PATCH 065/149] KvCoderComperator: set the offset correctly --- .../flink/dataflow/translation/types/KvCoderComperator.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index 30868fb53cc5..810958554d18 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -218,9 +218,11 @@ public void putNormalizedKey(KV record, MemorySegment target, int offset, final byte[] data = buffer1.getBuffer(); final int limit = offset + numBytes; - target.put(offset, data, 0, Math.min(numBytes, buffer1.size())); + int numBytesPut = Math.min(numBytes, buffer1.size()); - offset += buffer1.size(); + target.put(offset, data, 0, numBytesPut); + + offset += numBytesPut; while (offset < limit) { target.put(offset++, (byte) 0); From 11e08c09de81c1eaea2c1e2134fc73f4ee101f5c Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 23 Mar 2015 16:32:44 +0100 Subject: [PATCH 066/149] KvCoderComperator: set the correct context for the encoding --- .../flink/dataflow/translation/types/KvCoderComperator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index 810958554d18..eabbb8b6fb48 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -211,7 +211,7 @@ public boolean isNormalizedKeyPrefixOnly(int keyBytes) { public void putNormalizedKey(KV record, MemorySegment target, int offset, int numBytes) { buffer1.reset(); try { - keyCoder.encode(record.getKey(), buffer1, Coder.Context.OUTER); + keyCoder.encode(record.getKey(), buffer1, Coder.Context.NESTED); } catch (IOException e) { throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e); } From 402d0ee33b78c28f6d2c85ddb8b70e0ac155b203 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 23 Mar 2015 16:54:17 +0100 Subject: [PATCH 067/149] Maven: remove shade plugin Guava is now automatically shaded by Flink --- runners/flink/pom.xml | 24 ------------------------ 1 file changed, 24 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index ede8f5c20976..b4fd2faf5e0c 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -195,30 +195,6 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans - - - - - org.apache.maven.plugins - maven-shade-plugin - 2.3 - - - package - - shade - - - - - com.google - flink.dataflow.shaded.com.google - - - - - - From eea0dde5c7e972d475c62f5f2c188ca7dc6977cf Mon Sep 17 00:00:00 2001 From: Max Date: Tue, 24 Mar 2015 11:42:44 +0100 Subject: [PATCH 068/149] improve execution of WordCount example --- runners/flink/README.md | 8 ++++---- runners/flink/pom.xml | 26 +++++++++++++++++++++++++- 2 files changed, 29 insertions(+), 5 deletions(-) diff --git a/runners/flink/README.md b/runners/flink/README.md index aebfd1706bdd..fcb4cc4d1ab7 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -47,12 +47,12 @@ p.run(); ``` -To execute the example, get some sample data and run: - +To execute the example, let's first get some sample data: curl http://www.gutenberg.org/cache/epub/1128/pg1128.txt > kinglear.txt - mvn exec:java -Dexec.mainClass=com.dataartisans.flink.dataflow.examples.WordCount \ - -Dexec.args="--input=kinglear.txt --output=wordcounts.txt" +Then let's run the included WordCount locally on your machine: + + mvn exec:exec -Dinput=kinglear.txt -Doutput=wordcounts.txt Congratulations, you have run your first Google Dataflow program on top of Apache Flink! diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index b4fd2faf5e0c..d7f70edd3eeb 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -39,7 +39,13 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans UTF-8 UTF-8 - 0.9-SNAPSHOT + 0.9-SNAPSHOT + + + com.dataartisans.flink.dataflow.examples.WordCount + kinglear.txt + wordcounts.txt + 1 @@ -195,6 +201,24 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + java + + -classpath + + ${clazz} + --input=${input} + --output=${output} + --parallelism=${parallelism} + + + + From dee99230a06785c8d3c5dcb88615b6a720b4c6ef Mon Sep 17 00:00:00 2001 From: Max Date: Tue, 24 Mar 2015 11:51:59 +0100 Subject: [PATCH 069/149] README: add additional note about cluster setup --- runners/flink/README.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/runners/flink/README.md b/runners/flink/README.md index fcb4cc4d1ab7..8b7c8aacce60 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -56,3 +56,11 @@ Then let's run the included WordCount locally on your machine: mvn exec:exec -Dinput=kinglear.txt -Doutput=wordcounts.txt Congratulations, you have run your first Google Dataflow program on top of Apache Flink! + + +# Running Dataflow on Flink on a cluster + +You can run your Dataflow program on a Apache Flink cluster as well. For more +information, please visit the [Apache Flink Website](http://flink.apache.org) or +contact the +[Mailinglists](http://flink.apache.org/community.html#mailing-lists). From 8a9eb970eadb13cabba8503b88092988dcc36373 Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 25 Mar 2015 17:33:50 +0100 Subject: [PATCH 070/149] Flink master changes: rename FlatCombineFunction to GroupCombineFunction --- .../translation/functions/FlinkPartialReduceFunction.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java index ecb897f32872..3837ffae9d6a 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java @@ -19,19 +19,19 @@ import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.values.KV; -import org.apache.flink.api.common.functions.FlatCombineFunction; +import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.util.Collector; import java.util.Iterator; /** - * Flink {@link org.apache.flink.api.common.functions.FlatCombineFunction} for executing a + * Flink {@link org.apache.flink.api.common.functions.GroupCombineFunction} for executing a * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input * {@link com.google.cloud.dataflow.sdk.values.KV} elements VI, extracts the key and emits accumulated * values which have the intermediate format VA. */ -public class FlinkPartialReduceFunction implements FlatCombineFunction, KV> { +public class FlinkPartialReduceFunction implements GroupCombineFunction, KV> { private final Combine.KeyedCombineFn keyedCombineFn; From 6d25f9c3c82198bf5fd4bd63c4e91cb9f64432a8 Mon Sep 17 00:00:00 2001 From: Max Date: Tue, 7 Apr 2015 10:22:40 +0200 Subject: [PATCH 071/149] adapt Flink Dataflow to run with the latest DataflowJavaSDK --- .../flink/dataflow/examples/SideInputTest.java | 2 +- .../translation/FlinkTransformTranslators.java | 10 +++++----- .../flink/dataflow/translation/TranslationContext.java | 6 +++--- .../translation/functions/FlinkDoFnFunction.java | 2 +- .../functions/FlinkMultiOutputDoFnFunction.java | 2 +- .../translation/wrappers/SourceInputFormat.java | 2 +- .../dataartisans/flink/dataflow/ReadSourceITCase.java | 7 ++++++- 7 files changed, 18 insertions(+), 13 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java index e83347a3e044..a295223cf658 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java @@ -39,7 +39,7 @@ public static void main(String[] args) throws IOException, URISyntaxException { Pipeline p = Pipeline.create(options); - final PCollectionView totalDocuments = p + final PCollectionView totalDocuments = p .apply(Create.of("Hello!")) .apply(View.asSingleton()); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 98958c6d68bd..c8e7c30d5a36 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -433,11 +433,11 @@ public void translateNode(Flatten.FlattenPCollectionList transform, Translati } } - private static class CreatePCollectionViewTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class CreatePCollectionViewTranslator implements FlinkPipelineTranslator.TransformTranslator> { @Override - public void translateNode(View.CreatePCollectionView transform, TranslationContext context) { + public void translateNode(View.CreatePCollectionView transform, TranslationContext context) { DataSet inputDataSet = context.getInputDataSet(transform.getInput()); - PCollectionView input = transform.apply(null); + PCollectionView input = transform.apply(null); context.setSideInputDataSet(input, inputDataSet); } } @@ -473,11 +473,11 @@ public void translateNode(Create transform, TranslationContext context) { } } - private static void transformSideInputs(List> sideInputs, + private static void transformSideInputs(List> sideInputs, MapPartitionOperator outputDataSet, TranslationContext context) { // get corresponding Flink broadcast DataSets - for(PCollectionView input : sideInputs) { + for(PCollectionView input : sideInputs) { DataSet broadcastSet = context.getSideInputDataSet(input); outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId()); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index 887d94016d85..2df5b84e57d3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -36,7 +36,7 @@ public class TranslationContext { private final Map> dataSets; - private final Map, DataSet> broadcastDataSets; + private final Map, DataSet> broadcastDataSets; private final ExecutionEnvironment env; private final PipelineOptions options; @@ -72,11 +72,11 @@ public void setOutputDataSet(PValue value, DataSet set) { } @SuppressWarnings("unchecked") - public DataSet getSideInputDataSet(PCollectionView value) { + public DataSet getSideInputDataSet(PCollectionView value) { return (DataSet) broadcastDataSets.get(value); } - public void setSideInputDataSet(PCollectionView value, DataSet set) { + public void setSideInputDataSet(PCollectionView value, DataSet set) { if (!broadcastDataSets.containsKey(value)) { broadcastDataSets.put(value, set); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java index e0c771a293f0..5f833e805306 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java @@ -115,7 +115,7 @@ public PipelineOptions getPipelineOptions() { } @Override - public T sideInput(PCollectionView view) { + public T sideInput(PCollectionView view) { List sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal().getId()); List> windowedValueList = new ArrayList<>(sideInput.size()); for (T input : sideInput) { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java index 95bdadf0776d..80fb35589e6e 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -124,7 +124,7 @@ public PipelineOptions getPipelineOptions() { } @Override - public T sideInput(PCollectionView view) { + public T sideInput(PCollectionView view) { List sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal() .getId()); List> windowedValueList = new ArrayList<>(sideInput.size()); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java index 11d4c75bac8d..955dceef0dbc 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java @@ -117,7 +117,7 @@ public SourceInputSplit[] createInputSplits(int numSplits) throws IOException long desiredSizeBytes = 10000; try { desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits; - List> shards = initialSource.splitIntoShards(desiredSizeBytes, + List> shards = initialSource.splitIntoBundles(desiredSizeBytes, options); List> splits = Lists.newArrayList(); int splitCount = 0; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java index 08a63b6b911b..fb68194b55b1 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java @@ -89,7 +89,7 @@ class Read extends Source { } @Override - public List splitIntoShards(long desiredShardSizeBytes, PipelineOptions options) + public List splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options) throws Exception { List res = new ArrayList<>(); DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); @@ -131,6 +131,11 @@ public RangeReader(Read source) { this.current = source.from - 1; } + @Override + public boolean start() throws IOException { + return true; + } + @Override public boolean advance() throws IOException { current++; From ed9b10ea49a692a5d724574155790edd7be3834b Mon Sep 17 00:00:00 2001 From: Max Date: Tue, 7 Apr 2015 10:30:39 +0200 Subject: [PATCH 072/149] instead of using LATEST, pin to a version of the DataflowJavaSDK --- runners/flink/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index d7f70edd3eeb..6dc0419e6296 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -105,7 +105,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans com.google.cloud.dataflow google-cloud-dataflow-java-examples-all - LATEST + 0.3.150326 org.slf4j From 0d1241922ac95770d06562e9d5bd4a72eac0d95d Mon Sep 17 00:00:00 2001 From: mbalassi Date: Thu, 9 Apr 2015 14:33:39 +0200 Subject: [PATCH 073/149] Pinned version for SDK too --- runners/flink/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 6dc0419e6296..217988b23213 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -93,7 +93,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans com.google.cloud.dataflow google-cloud-dataflow-java-sdk-all - LATEST + 0.3.150326 org.slf4j From a9329704648c0006e1402e6077691154d4568a8b Mon Sep 17 00:00:00 2001 From: mbalassi Date: Mon, 20 Apr 2015 16:23:07 +0200 Subject: [PATCH 074/149] Updated Aggregator wrappers to match new Flink Accumulator behaviour This might lead to runtime failure when the fields of the aggregator wrapper are not java serializable. --- .../wrappers/CombineFnAggregatorWrapper.java | 22 +------------------ .../SerializableFnAggregatorWrapper.java | 22 ------------------- 2 files changed, 1 insertion(+), 43 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java index 16e2bdcc7c04..10ce8c0f0576 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java @@ -84,25 +84,5 @@ public Accumulator clone() { public void addValue(AI value) { add(value); } - - @Override - public void write(ObjectOutputStream out) throws IOException { - byte[] aaByte = SerializableUtils.serializeToByteArray((Serializable) aa); - byte[] combinerByte = SerializableUtils.serializeToByteArray(combiner); - out.write(aaByte.length); - out.write(aaByte); - out.write(combinerByte.length); - out.write(combinerByte); - } - - @Override - @SuppressWarnings("unchecked") - public void read(ObjectInputStream in) throws IOException { - byte[] aaByte = new byte[in.readInt()]; - in.read(aaByte); - byte[] combinerByte = new byte[in.readInt()]; - in.read(combinerByte); - this.aa = (AA) SerializableUtils.deserializeFromByteArray(aaByte, "AggreatorValue"); - this.combiner = (Combine.CombineFn) SerializableUtils.deserializeFromByteArray(combinerByte, "AggreatorCombiner"); - } + } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java index acf9a4d98451..539c195bef6d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -75,28 +75,6 @@ public void addValue(AI value) { add(value); } - @Override - public void write(ObjectOutputStream out) throws IOException { - byte[] aaByte = SerializableUtils.serializeToByteArray((Serializable) aa); - byte[] combinerByte = SerializableUtils.serializeToByteArray(serFun); - out.writeInt(aaByte.length); - out.write(aaByte); - out.writeInt(combinerByte.length); - out.write(combinerByte); - } - - @Override - @SuppressWarnings("unchecked") - public void read(ObjectInputStream in) throws IOException { - byte[] aaByte = new byte[in.readInt()]; - in.read(aaByte); - byte[] serFunByte = new byte[in.readInt()]; - in.read(serFunByte); - this.aa = (AO) SerializableUtils.deserializeFromByteArray(aaByte, "AggreatorValue"); - this.serFun = (SerializableFunction, AO>) SerializableUtils.deserializeFromByteArray(serFunByte, "AggreatorSerializableFunction"); - - } - @Override public Accumulator clone() { // copy it by merging From f0097cce3d313a0c4ff38b7f2f416fa600923610 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 4 Jun 2015 10:47:20 +0200 Subject: [PATCH 075/149] adapt to the latest Flink master --- .../flink/dataflow/FlinkPipelineRunner.java | 8 ++++---- .../translation/FlinkTransformTranslators.java | 2 +- .../translation/wrappers/SourceInputSplit.java | 17 ----------------- 3 files changed, 5 insertions(+), 22 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java index ed553ca87998..7ea685d849a4 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java @@ -130,7 +130,7 @@ private ExecutionEnvironment createExecutionEnvironment(FlinkPipelineOptions opt if (masterUrl.equals("[local]")) { ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); if (options.getParallelism() != -1) { - env.setDegreeOfParallelism(options.getParallelism()); + env.setParallelism(options.getParallelism()); } return env; } else if (masterUrl.equals("[collection]")) { @@ -138,7 +138,7 @@ private ExecutionEnvironment createExecutionEnvironment(FlinkPipelineOptions opt } else if (masterUrl.equals("[auto]")) { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); if (options.getParallelism() != -1) { - env.setDegreeOfParallelism(options.getParallelism()); + env.setParallelism(options.getParallelism()); } return env; } else if (masterUrl.matches(".*:\\d*")) { @@ -148,14 +148,14 @@ private ExecutionEnvironment createExecutionEnvironment(FlinkPipelineOptions opt Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()])); if (options.getParallelism() != -1) { - env.setDegreeOfParallelism(options.getParallelism()); + env.setParallelism(options.getParallelism()); } return env; } else { LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl); ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); if (options.getParallelism() != -1) { - env.setDegreeOfParallelism(options.getParallelism()); + env.setParallelism(options.getParallelism()); } return env; } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index c8e7c30d5a36..039804ff3712 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -253,7 +253,7 @@ private static class ConsoleIOWriteTranslator implements FlinkPipelineTranslator @Override public void translateNode(ConsoleIO.Write.Bound transform, TranslationContext context) { DataSet inputDataSet = context.getInputDataSet(transform.getInput()); - inputDataSet.print().name(transform.getName()); + inputDataSet.printOnTaskManager(transform.getName()); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java index 0c5165e9531c..8dc75735be61 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java @@ -54,21 +54,4 @@ public Source getSource() { return source; } - @Override - public void write(DataOutputView out) throws IOException { - out.writeInt(splitNumber); - byte[] serializedSource = SerializableUtils.serializeToByteArray(source); - out.writeInt(serializedSource.length); - out.write(serializedSource); - } - - @Override - @SuppressWarnings("unchecked") - public void read(DataInputView in) throws IOException { - splitNumber = in.readInt(); - int length = in.readInt(); - byte[] serializedSource = new byte[length]; - in.read(serializedSource, 0, length); - source = (Source) SerializableUtils.deserializeFromByteArray(serializedSource, "Input Source"); - } } From 1f4d2f5da8d338482908710771cad31d2721935b Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 4 Jun 2015 10:59:23 +0200 Subject: [PATCH 076/149] [streaming] add a hint to the streaming branch in the readme --- runners/flink/README.md | 4 ++++ .../com/dataartisans/flink/dataflow/FlinkPipelineOptions.java | 3 ++- .../com/dataartisans/flink/dataflow/FlinkPipelineRunner.java | 4 ++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/runners/flink/README.md b/runners/flink/README.md index 8b7c8aacce60..a4b1d7eb69ec 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -64,3 +64,7 @@ You can run your Dataflow program on a Apache Flink cluster as well. For more information, please visit the [Apache Flink Website](http://flink.apache.org) or contact the [Mailinglists](http://flink.apache.org/community.html#mailing-lists). + +# Streaming + +Streaming support is currently under development. See the `streaming` branch for the current version. diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java index 7a7f2402598e..fdd4e5930b62 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java @@ -21,13 +21,14 @@ import com.google.cloud.dataflow.sdk.options.Default; import com.google.cloud.dataflow.sdk.options.Description; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.StreamingOptions; import java.util.List; /** * Options which can be used to configure a Flink PipelineRunner. */ -public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOptions { +public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOptions, StreamingOptions { /** * List of local files to make available to workers. diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java index 7ea685d849a4..3636783081ef 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java @@ -109,6 +109,10 @@ public static FlinkPipelineRunner fromOptions(PipelineOptions options) { flinkOptions.setFlinkMaster("[auto]"); } + if (flinkOptions.isStreaming()) { + throw new RuntimeException("Streaming is currently not supported."); + } + return new FlinkPipelineRunner(flinkOptions); } From 19ddb2dfbb2d5b161e6defe37fc0b07a605eeee0 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 4 Jun 2015 11:06:05 +0200 Subject: [PATCH 077/149] add a Travis config file to enable continuous integration --- runners/flink/.travis.yml | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 runners/flink/.travis.yml diff --git a/runners/flink/.travis.yml b/runners/flink/.travis.yml new file mode 100644 index 000000000000..14e95e9d971a --- /dev/null +++ b/runners/flink/.travis.yml @@ -0,0 +1,10 @@ +sudo: false + +language: java + +matrix: + include: + - jdk: "oraclejdk7" + - jdk: "oraclejdk8" + - jdk: "openjdk6" + - jdk: "openjdk7" From 0a9a77635bca0feff3f46a432b5c962f821865c4 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 4 Jun 2015 11:29:08 +0200 Subject: [PATCH 078/149] travis: modify config file to adjust jdk version and test command --- runners/flink/.travis.yml | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/runners/flink/.travis.yml b/runners/flink/.travis.yml index 14e95e9d971a..d8921b9ec575 100644 --- a/runners/flink/.travis.yml +++ b/runners/flink/.travis.yml @@ -2,9 +2,10 @@ sudo: false language: java -matrix: - include: - - jdk: "oraclejdk7" - - jdk: "oraclejdk8" - - jdk: "openjdk6" - - jdk: "openjdk7" +jdk: + - oraclejdk7 + - oraclejdk8 + - openjdk7 + - openjdk8 + +script: mvn verify From 79bf260585bb13fba6011a4849204ca81ae39fd3 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 4 Jun 2015 11:33:36 +0200 Subject: [PATCH 079/149] travis: remove openjdk8 since it is not supported --- runners/flink/.travis.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/runners/flink/.travis.yml b/runners/flink/.travis.yml index d8921b9ec575..d7fbf5be9438 100644 --- a/runners/flink/.travis.yml +++ b/runners/flink/.travis.yml @@ -6,6 +6,7 @@ jdk: - oraclejdk7 - oraclejdk8 - openjdk7 - - openjdk8 + # not supported yet :( + #- openjdk8 script: mvn verify From b4092a72a1452673c8a2b70f82c8a797ac08a15f Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 1 Oct 2015 17:34:48 +0200 Subject: [PATCH 080/149] bump Flink version to 0.9.1 --- runners/flink/pom.xml | 2 +- .../flink/dataflow/translation/types/KvCoderComperator.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 217988b23213..baf7f288d9ac 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -39,7 +39,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans UTF-8 UTF-8 - 0.9-SNAPSHOT + 0.9.1 com.dataartisans.flink.dataflow.examples.WordCount diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index eabbb8b6fb48..53d63d7b2401 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -130,7 +130,8 @@ public int compareToReference(TypeComparator> other) { return arr[i] - arrOther[i]; } } - return 0; } + return 0; + } @Override From fbbba8607489eaf1bff76a5bc1722085312b004d Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 9 Oct 2015 12:26:12 +0200 Subject: [PATCH 081/149] change version number to 0.1 --- runners/flink/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index baf7f288d9ac..2443bb04ab32 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -21,7 +21,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans com.dataartisans flink-dataflow - 1-SNAPSHOT + 0.1 Flink Dataflow Runner jar From fc2a25ac0afa24346567cbd0a1e0e539f6c9aacf Mon Sep 17 00:00:00 2001 From: Max Date: Sun, 11 Oct 2015 15:16:48 +0200 Subject: [PATCH 082/149] remove unused serializer --- .../flink/dataflow/translation/types/KvCoderComperator.java | 1 - 1 file changed, 1 deletion(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index 53d63d7b2401..887452ed4345 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -160,7 +160,6 @@ public int compare(KV first, KV second) { @Override public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { - CoderTypeSerializer> serializer = new CoderTypeSerializer>(coder); inputWrapper.setInputView(firstSource); K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); inputWrapper.setInputView(secondSource); From b8dcf4793d3ee1e5fcaab25d7cb311efa51711dc Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 5 Oct 2015 11:35:08 +0200 Subject: [PATCH 083/149] update to Dataflow SDK 1.0.0 --- runners/flink/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 2443bb04ab32..abccc77239e8 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -89,11 +89,11 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans ${flink.version} test - + com.google.cloud.dataflow google-cloud-dataflow-java-sdk-all - 0.3.150326 + 1.0.0 org.slf4j @@ -105,7 +105,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans com.google.cloud.dataflow google-cloud-dataflow-java-examples-all - 0.3.150326 + 1.0.0 org.slf4j From a2630800f620bf592f6aaedfaadfa26eceb220e0 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 5 Oct 2015 11:35:40 +0200 Subject: [PATCH 084/149] fix FlinkRunnerResult for new PipelineResult --- .../flink/dataflow/FlinkRunnerResult.java | 31 ++++++++++++++----- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java index 56cb297da8a9..92185e234e21 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java @@ -16,6 +16,11 @@ package com.dataartisans.flink.dataflow; import com.google.cloud.dataflow.sdk.PipelineResult; +import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException; +import com.google.cloud.dataflow.sdk.runners.AggregatorValues; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsNotFound; + import java.util.Collections; import java.util.Map; @@ -38,15 +43,25 @@ public FlinkRunnerResult(Map aggregators, long runtime) { this.runtime = runtime; } - /** - * Return the final values of all {@link com.google.cloud.dataflow.sdk.transforms.Aggregator}s - * used in the {@link com.google.cloud.dataflow.sdk.Pipeline}. - */ - public Map getAggregators() { - return aggregators; + @Override + public State getState() { + return null; } - public long getRuntime() { - return runtime; + @Override + public AggregatorValues getAggregatorValues(final Aggregator aggregator) throws AggregatorRetrievalException { + // TODO provide a list of all accumulator step values + Object value = aggregators.get(aggregator.getName()); + if (value != null) { + return new AggregatorValues() { + @Override + public Map getValuesAtSteps() { + return (Map) aggregators; + } + }; + } else { + throw new AggregatorRetrievalException("Accumulator results not found.", + new RuntimeException("Accumulator does not exist.")); + } } } From 72cb4f185290601e00edece8a58c4b93bba32054 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 5 Oct 2015 11:38:59 +0200 Subject: [PATCH 085/149] fix UnionCoder --- .../translation/functions/UnionCoder.java | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java index a246a52c4743..da5e6dc6c3a3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java @@ -101,6 +101,13 @@ public List> getCoderArguments() { return null; } + @Override + public void verifyDeterministic() throws NonDeterministicException { + for (Coder coder : elementCoders) { + coder.verifyDeterministic(); + } + } + @Override public List> getComponents() { return elementCoders; @@ -141,15 +148,4 @@ public void registerByteSizeObserver( private UnionCoder(List> elementCoders) { this.elementCoders = elementCoders; } - - @Override - public boolean isDeterministic() { - for (Coder elementCoder : elementCoders) { - if (!elementCoder.isDeterministic()) { - return false; - } - } - - return true; - } } From defc7c932910704ef18422176761a99bc6b07164 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 8 Oct 2015 17:16:28 +0200 Subject: [PATCH 086/149] adapt to 1.0.0 changes (handle AppliedPTransform) --- .../flink/dataflow/examples/WordCount.java | 2 +- .../translation/FlinkPipelineTranslator.java | 12 ++++++++--- .../translation/TranslationContext.java | 21 ++++++++++++++++++- 3 files changed, 30 insertions(+), 5 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java index 9521d5579c89..6fd04891ae6e 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java @@ -52,7 +52,7 @@ public static interface Options extends PipelineOptions, FlinkPipelineOptions { } public static void main(String[] args) { - + Options options = PipelineOptionsFactory.fromArgs(args).as(Options.class); options.setRunner(FlinkPipelineRunner.class); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java index 8d2ece2918f5..855563de3943 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java @@ -21,9 +21,12 @@ import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; import com.google.cloud.dataflow.sdk.values.PValue; import org.apache.flink.api.java.ExecutionEnvironment; @@ -77,8 +80,7 @@ public void enterCompositeTransform(TransformTreeNode node) { if (translator != null) { inComposite = true; - if (transform instanceof CoGroupByKey && - ((CoGroupByKey) transform).getInput().getKeyedCollections().size() != 2) { + if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) { // we can only optimize CoGroupByKey for input size 2 inComposite = false; } @@ -144,13 +146,17 @@ public void visitValue(PValue value, TransformTreeNode producer) { @SuppressWarnings("unchecked") TransformTranslator typedTranslator = (TransformTranslator) translator; + // create the applied PTransform on the context + context.setCurrentTransform(AppliedPTransform.of( + node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); + typedTranslator.translateNode(typedTransform, context); } /** * A translator of a {@link PTransform}. */ - public static interface TransformTranslator { + public interface TransformTranslator { void translateNode(Type transform, TranslationContext context); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index 2df5b84e57d3..fc149aaf175c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.PValue; import com.google.cloud.dataflow.sdk.values.TypedPValue; @@ -37,9 +38,11 @@ public class TranslationContext { private final Map> dataSets; private final Map, DataSet> broadcastDataSets; - + private final ExecutionEnvironment env; private final PipelineOptions options; + + private AppliedPTransform currentTransform; // ------------------------------------------------------------------------ @@ -71,6 +74,22 @@ public void setOutputDataSet(PValue value, DataSet set) { } } + /** + * Gets the applied AppliedPTransform which carries input/output. + * @return + */ + public AppliedPTransform getCurrentTransform() { + return currentTransform; + } + + /** + * Sets the AppliedPTransform which carries input/output. + * @param currentTransform + */ + public void setCurrentTransform(AppliedPTransform currentTransform) { + this.currentTransform = currentTransform; + } + @SuppressWarnings("unchecked") public DataSet getSideInputDataSet(PCollectionView value) { return (DataSet) broadcastDataSets.get(value); From 893ffde4d8896a5d3c1e535fdfa66ad303b7a50e Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 8 Oct 2015 17:36:22 +0200 Subject: [PATCH 087/149] fix renaming of ReadSource to Read.Bounded --- .../translation/FlinkTransformTranslators.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 039804ff3712..ea72a4451625 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -26,7 +26,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.io.AvroIO; -import com.google.cloud.dataflow.sdk.io.ReadSource; +import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.Source; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.transforms.Combine; @@ -87,7 +87,7 @@ public class FlinkTransformTranslators { // we don't need this because we translate the Combine.PerKey directly // TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator()); - TRANSLATORS.put(Create.class, new CreateTranslator()); + TRANSLATORS.put(Create.Values.class, new CreateTranslator()); TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslator()); TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator()); @@ -104,7 +104,8 @@ public class FlinkTransformTranslators { //TRANSLATORS.put(PubsubIO.Read.Bound.class, null); //TRANSLATORS.put(PubsubIO.Write.Bound.class, null); - TRANSLATORS.put(ReadSource.Bound.class, new ReadSourceTranslator()); + TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslator()); +// TRANSLATORS.put(Write.Bound.class, new ReadSourceTranslator()); TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslator()); TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslator()); @@ -120,10 +121,10 @@ public static FlinkPipelineTranslator.TransformTranslator getTranslator(PTran return TRANSLATORS.get(transform.getClass()); } - private static class ReadSourceTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class ReadSourceTranslator implements FlinkPipelineTranslator.TransformTranslator> { @Override - public void translateNode(ReadSource.Bound transform, TranslationContext context) { + public void translateNode(Read.Bounded transform, TranslationContext context) { String name = transform.getName(); Source source = transform.getSource(); Coder coder = transform.getOutput().getCoder(); From d467af4bacd3662639f5dadc9998453bbac9802c Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 8 Oct 2015 18:23:05 +0200 Subject: [PATCH 088/149] adapt to the API changes: get types from context --- .../FlinkTransformTranslators.java | 121 +++++++++--------- .../translation/TranslationContext.java | 37 ++++++ 2 files changed, 101 insertions(+), 57 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index ea72a4451625..542c35c5123d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -23,6 +23,7 @@ import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation; import com.dataartisans.flink.dataflow.translation.wrappers.SourceInputFormat; import com.google.api.client.util.Maps; +import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.io.AvroIO; @@ -45,6 +46,8 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.PValue; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.Lists; import org.apache.avro.Schema; @@ -104,7 +107,7 @@ public class FlinkTransformTranslators { //TRANSLATORS.put(PubsubIO.Read.Bound.class, null); //TRANSLATORS.put(PubsubIO.Write.Bound.class, null); - TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslator()); +// TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslator()); // TRANSLATORS.put(Write.Bound.class, new ReadSourceTranslator()); TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslator()); @@ -121,21 +124,22 @@ public static FlinkPipelineTranslator.TransformTranslator getTranslator(PTran return TRANSLATORS.get(transform.getClass()); } - private static class ReadSourceTranslator implements FlinkPipelineTranslator.TransformTranslator> { - - @Override - public void translateNode(Read.Bounded transform, TranslationContext context) { - String name = transform.getName(); - Source source = transform.getSource(); - Coder coder = transform.getOutput().getCoder(); - - TypeInformation typeInformation = context.getTypeInfo(transform.getOutput()); - - DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), new SourceInputFormat<>(source, context.getPipelineOptions(), coder), typeInformation, name); - - context.setOutputDataSet(transform.getOutput(), dataSource); - } - } +// private static class ReadSourceTranslator implements FlinkPipelineTranslator.TransformTranslator> { +// +// @Override +// public void translateNode(Read.Bounded transform, TranslationContext context) { +// String name = transform.getName(); +// Source source = transform.getSource(); +// PValue output = context.getCurrentTransform().getOutput(). +// Coder coder = output.getCoder(); +// +// TypeInformation typeInformation = context.getTypeInfo(output); +// +// DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), new SourceInputFormat<>(source, context.getPipelineOptions(), coder), typeInformation, name); +// +// context.setOutputDataSet(output, dataSource); +// } +// } private static class AvroIOReadTranslator implements FlinkPipelineTranslator.TransformTranslator> { private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslator.class); @@ -144,9 +148,10 @@ private static class AvroIOReadTranslator implements FlinkPipelineTranslator. public void translateNode(AvroIO.Read.Bound transform, TranslationContext context) { String path = transform.getFilepattern(); String name = transform.getName(); - Schema schema = transform.getSchema(); +// Schema schema = transform.getSchema(); + PValue output = context.getCurrentOutput(); - TypeInformation typeInformation = context.getTypeInfo(transform.getOutput()); + TypeInformation typeInformation = context.getCurrentInputTypeInfo(); // This is super hacky, but unfortunately we cannot get the type otherwise Class avroType = null; @@ -161,7 +166,7 @@ public void translateNode(AvroIO.Read.Bound transform, TranslationContext con DataSource source = new DataSource<>(context.getExecutionEnvironment(), new AvroInputFormat<>(new Path(path), avroType), typeInformation, name); - context.setOutputDataSet(transform.getOutput(), source); + context.setOutputDataSet(output, source); } } @@ -170,7 +175,7 @@ private static class AvroIOWriteTranslator implements FlinkPipelineTranslator @Override public void translateNode(AvroIO.Write.Bound transform, TranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(transform.getInput()); + DataSet inputDataSet = context.getCurrentInputDataSet(); String filenamePrefix = transform.getFilenamePrefix(); String filenameSuffix = transform.getFilenameSuffix(); int numShards = transform.getNumShards(); @@ -216,11 +221,13 @@ public void translateNode(TextIO.Read.Bound transform, TranslationContex LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType); LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation); - TypeInformation typeInformation = context.getTypeInfo(transform.getOutput()); + PValue output = (PValue) context.getCurrentTransform().getOutput(); + + TypeInformation typeInformation = context.getTypeInfo(output); DataSource source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name); - context.setOutputDataSet(transform.getOutput(), source); + context.setOutputDataSet(output, source); } } @@ -229,7 +236,9 @@ private static class TextIOWriteTranslator implements FlinkPipelineTranslator @Override public void translateNode(TextIO.Write.Bound transform, TranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(transform.getInput()); + PValue input = (PValue) context.getCurrentTransform().getInput(); + DataSet inputDataSet = context.getInputDataSet(input); + String filenamePrefix = transform.getFilenamePrefix(); String filenameSuffix = transform.getFilenameSuffix(); boolean needsValidation = transform.needsValidation(); @@ -253,7 +262,8 @@ public void translateNode(TextIO.Write.Bound transform, TranslationContext co private static class ConsoleIOWriteTranslator implements FlinkPipelineTranslator.TransformTranslator { @Override public void translateNode(ConsoleIO.Write.Bound transform, TranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(transform.getInput()); + PValue input = (PValue) context.getCurrentTransform().getInput(); + DataSet inputDataSet = context.getInputDataSet(input); inputDataSet.printOnTaskManager(transform.getName()); } } @@ -262,16 +272,16 @@ private static class GroupByKeyOnlyTranslator implements FlinkPipelineTran @Override public void translateNode(GroupByKey.GroupByKeyOnly transform, TranslationContext context) { - DataSet> inputDataSet = context.getInputDataSet(transform.getInput()); + DataSet> inputDataSet = context.getCurrentInputDataSet(); GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); - TypeInformation>> typeInformation = context.getTypeInfo(transform.getOutput()); + TypeInformation>> typeInformation = context.getCurrentInputTypeInfo(); Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); GroupReduceOperator, KV>> outputDataSet = new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); - context.setOutputDataSet(transform.getOutput(), outputDataSet); + context.setOutputDataSet(context.getCurrentOutput(), outputDataSet); } } @@ -279,16 +289,21 @@ private static class CombinePerKeyTranslator implements FlinkPipe @Override public void translateNode(Combine.PerKey transform, TranslationContext context) { - DataSet> inputDataSet = context.getInputDataSet(transform.getInput()); + DataSet> inputDataSet = context.getCurrentInputDataSet(); @SuppressWarnings("unchecked") Combine.KeyedCombineFn keyedCombineFn = (Combine.KeyedCombineFn) transform.getFn(); - KvCoder inputCoder = (KvCoder) transform.getInput().getCoder(); + KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); Coder accumulatorCoder = - keyedCombineFn.getAccumulatorCoder(transform.getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder()); - + null; + try { + accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder()); + } catch (CannotProvideCoderException e) { + e.printStackTrace(); + // TODO + } TypeInformation> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder); TypeInformation> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder)); @@ -305,7 +320,7 @@ public void translateNode(Combine.PerKey transform, TranslationContex // Reduce fully to VO GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); - TypeInformation> reduceTypeInfo = context.getTypeInfo(transform.getOutput()); + TypeInformation> reduceTypeInfo = context.getTypeInfo(context.getCurrentOutput()); Grouping> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); @@ -313,7 +328,7 @@ public void translateNode(Combine.PerKey transform, TranslationContex GroupReduceOperator, KV> outputDataSet = new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName()); - context.setOutputDataSet(transform.getOutput(), outputDataSet); + context.setOutputDataSet(context.getCurrentOutput(), outputDataSet); } } @@ -342,22 +357,18 @@ private static class ParDoBoundTranslator implements FlinkPipelineTrans @Override public void translateNode(ParDo.Bound transform, TranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(transform.getInput()); + DataSet inputDataSet = context.getInputDataSet(context.getCurrentInput()); final DoFn doFn = transform.getFn(); - if (doFn instanceof DoFn.RequiresKeyedState) { - LOG.error("Flink Batch Execution does not support Keyed State."); - } - - TypeInformation typeInformation = context.getTypeInfo(transform.getOutput()); + TypeInformation typeInformation = context.getTypeInfo(context.getCurrentOutput()); FlinkDoFnFunction doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions()); MapPartitionOperator outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName()); transformSideInputs(transform.getSideInputs(), outputDataSet, context); - context.setOutputDataSet(transform.getOutput(), outputDataSet); + context.setOutputDataSet(context.getCurrentOutput(), outputDataSet); } } @@ -366,15 +377,11 @@ private static class ParDoBoundMultiTranslator implements FlinkPipeline @Override public void translateNode(ParDo.BoundMulti transform, TranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(transform.getInput()); + DataSet inputDataSet = context.getInputDataSet(context.getCurrentInput()); final DoFn doFn = transform.getFn(); - if (doFn instanceof DoFn.RequiresKeyedState) { - LOG.error("Flink Batch Execution does not support Keyed State."); - } - - Map, PCollection> outputs = transform.getOutput().getAll(); + Map, PCollection> outputs = context.getOutput(transform).getAll(); Map, Integer> outputMap = Maps.newHashMap(); // put the main output at index 0, FlinkMultiOutputDoFnFunction also expects this @@ -420,7 +427,7 @@ private static class FlattenPCollectionTranslator implements FlinkPipelineTra @Override public void translateNode(Flatten.FlattenPCollectionList transform, TranslationContext context) { - List> allInputs = transform.getInput().getAll(); + List> allInputs = context.getInput(transform).getAll(); DataSet result = null; for(PCollection collection : allInputs) { DataSet current = context.getInputDataSet(collection); @@ -430,24 +437,24 @@ public void translateNode(Flatten.FlattenPCollectionList transform, Translati result = result.union(current); } } - context.setOutputDataSet(transform.getOutput(), result); + context.setOutputDataSet(context.getCurrentOutput(), result); } } private static class CreatePCollectionViewTranslator implements FlinkPipelineTranslator.TransformTranslator> { @Override public void translateNode(View.CreatePCollectionView transform, TranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(transform.getInput()); + DataSet inputDataSet = context.getInputDataSet(context.getCurrentInput()); PCollectionView input = transform.apply(null); context.setSideInputDataSet(input, inputDataSet); } } - private static class CreateTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class CreateTranslator implements FlinkPipelineTranslator.TransformTranslator> { @Override - public void translateNode(Create transform, TranslationContext context) { - TypeInformation typeInformation = context.getTypeInfo(transform.getOutput()); + public void translateNode(Create.Values transform, TranslationContext context) { + TypeInformation typeInformation = context.getCurrentOutputTypeInfo(); Iterable elements = transform.getElements(); // we need to serialize the elements to byte arrays, since they might contain @@ -455,7 +462,7 @@ public void translateNode(Create transform, TranslationContext context) { // in the FlatMap function using the Coder. List serializedElements = Lists.newArrayList(); - Coder coder = transform.getOutput().getCoder(); + Coder coder = context.getOutput(transform).getCoder(); for (OUT element: elements) { ByteArrayOutputStream bao = new ByteArrayOutputStream(); try { @@ -470,7 +477,7 @@ public void translateNode(Create transform, TranslationContext context) { FlinkCreateFunction flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder); FlatMapOperator outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName()); - context.setOutputDataSet(transform.getOutput(), outputDataSet); + context.setOutputDataSet(context.getCurrentOutput(), outputDataSet); } } @@ -493,7 +500,7 @@ private static class CoGroupByKeyTranslator implements FlinkPipelineT @Override public void translateNode(CoGroupByKey transform, TranslationContext context) { - KeyedPCollectionTuple input = transform.getInput(); + KeyedPCollectionTuple input = context.getInput(transform); CoGbkResultSchema schema = input.getCoGbkResultSchema(); List> keyedCollections = input.getKeyedCollections(); @@ -510,7 +517,7 @@ public void translateNode(CoGroupByKey transform, TranslationContext context) DataSet> inputDataSet1 = context.getInputDataSet(collection1); DataSet> inputDataSet2 = context.getInputDataSet(collection2); - TypeInformation> typeInfo = context.getTypeInfo(transform.getOutput()); + TypeInformation> typeInfo = context.getCurrentOutputTypeInfo(); FlinkCoGroupKeyedListAggregator aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2); @@ -520,7 +527,7 @@ public void translateNode(CoGroupByKey transform, TranslationContext context) DataSet> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2, keySelector1, keySelector2, aggregator, typeInfo, null, transform.getName()); - context.setOutputDataSet(transform.getOutput(), out); + context.setOutputDataSet(context.getCurrentOutput(), out); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index fc149aaf175c..7f2d8191ae96 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -23,7 +23,10 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; +import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; import com.google.cloud.dataflow.sdk.values.PValue; import com.google.cloud.dataflow.sdk.values.TypedPValue; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -34,6 +37,8 @@ import java.util.HashMap; import java.util.Map; +import static com.google.common.base.Preconditions.checkArgument; + public class TranslationContext { private final Map> dataSets; @@ -67,6 +72,10 @@ public PipelineOptions getPipelineOptions() { public DataSet getInputDataSet(PValue value) { return (DataSet) dataSets.get(value); } + + public DataSet getCurrentInputDataSet() { + return getInputDataSet(getCurrentInput()); + } public void setOutputDataSet(PValue value, DataSet set) { if (!dataSets.containsKey(value)) { @@ -113,4 +122,32 @@ public TypeInformation getTypeInfo(PValue output) { } return new GenericTypeInfo((Class)Object.class); } + + public TypeInformation getCurrentInputTypeInfo() { + return getTypeInfo((PValue) currentTransform.getInput()); + } + + public TypeInformation getCurrentOutputTypeInfo() { + return getTypeInfo((PValue) currentTransform.getOutput()); + } + + public PValue getCurrentInput() { + return (PValue) currentTransform.getInput(); + } + + public PValue getCurrentOutput() { + return (PValue) currentTransform.getOutput(); + } + + @SuppressWarnings("unchecked") + I getInput(PTransform transform) { + I input = (I) currentTransform.getInput(); + return input; + } + + @SuppressWarnings("unchecked") + O getOutput(PTransform transform) { + O output = (O) currentTransform.getOutput(); + return output; + } } From abcc06cdf217a8a36cb47df0764ce88ec492ac5b Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 8 Oct 2015 18:57:31 +0200 Subject: [PATCH 089/149] adapt to new windowing semantics --- .../functions/FlinkDoFnFunction.java | 36 +++++++++-------- .../FlinkMultiOutputDoFnFunction.java | 33 ++++++++-------- .../wrappers/CombineFnAggregatorWrapper.java | 14 ++++++- .../SerializableFnAggregatorWrapper.java | 39 ++++++++++--------- 4 files changed, 70 insertions(+), 52 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java index 5f833e805306..a1d3b984f553 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java @@ -25,10 +25,13 @@ import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowingInternals; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableList; +import javafx.scene.layout.Pane; import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.util.Collector; import org.joda.time.Instant; @@ -94,10 +97,6 @@ public IN element() { return this.inValue; } - @Override - public DoFn.KeyedState keyedState() { - throw new UnsupportedOperationException("Getting the keyed state is not supported!"); - } @Override public Instant timestamp() { @@ -105,8 +104,18 @@ public Instant timestamp() { } @Override - public Collection windows() { - return ImmutableList.of(); + public BoundedWindow window() { + return GlobalWindow.INSTANCE; + } + + @Override + public PaneInfo pane() { + return PaneInfo.NO_FIRING; + } + + @Override + public WindowingInternals windowingInternals() { + return null; } @Override @@ -119,7 +128,7 @@ public T sideInput(PCollectionView view) { List sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal().getId()); List> windowedValueList = new ArrayList<>(sideInput.size()); for (T input : sideInput) { - windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE))); + windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane())); } return view.fromIterableInternal(windowedValueList); } @@ -147,17 +156,12 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times } @Override - public Aggregator createAggregator(String name, Combine.CombineFn combiner) { - CombineFnAggregatorWrapper wrapper = new CombineFnAggregatorWrapper(combiner); + protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { + SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper(combiner); getRuntimeContext().addAccumulator(name, wrapper); - return wrapper; + return null; } - @Override - public Aggregator createAggregator(String name, SerializableFunction, AO> serializableFunction) { - SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper(serializableFunction); - getRuntimeContext().addAccumulator(name, wrapper); - return wrapper; - } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java index 80fb35589e6e..84f3e27975ec 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -26,7 +26,9 @@ import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowingInternals; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableList; @@ -104,18 +106,23 @@ public IN element() { } @Override - public DoFn.KeyedState keyedState() { - throw new UnsupportedOperationException("Getting the keyed state is not supported!"); + public Instant timestamp() { + return Instant.now(); } @Override - public Instant timestamp() { - return Instant.now(); + public BoundedWindow window() { + return GlobalWindow.INSTANCE; + } + + @Override + public PaneInfo pane() { + return PaneInfo.NO_FIRING; } @Override - public Collection windows() { - return ImmutableList.of(); + public WindowingInternals windowingInternals() { + return null; } @Override @@ -129,7 +136,7 @@ public T sideInput(PCollectionView view) { .getId()); List> windowedValueList = new ArrayList<>(sideInput.size()); for (T input : sideInput) { - windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE))); + windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane())); } return view.fromIterableInternal(windowedValueList); } @@ -161,17 +168,11 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times } @Override - public Aggregator createAggregator(String name, Combine.CombineFn combiner) { - CombineFnAggregatorWrapper wrapper = new CombineFnAggregatorWrapper<>(combiner); + protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { + SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper(combiner); getRuntimeContext().addAccumulator(name, wrapper); - return wrapper; + return null; } - @Override - public Aggregator createAggregator(String name, SerializableFunction, AO> serializableFunction) { - SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper<>(serializableFunction); - getRuntimeContext().addAccumulator(name, wrapper); - return wrapper; - } } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java index 10ce8c0f0576..651cfe629565 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java @@ -36,7 +36,7 @@ * the combine function as an aggregator in a {@link com.google.cloud.dataflow.sdk.transforms.ParDo} * operation. */ -public class CombineFnAggregatorWrapper implements Aggregator, Accumulator { +public class CombineFnAggregatorWrapper implements Aggregator, Accumulator { private AA aa; private Combine.CombineFn combiner; @@ -84,5 +84,15 @@ public Accumulator clone() { public void addValue(AI value) { add(value); } - + + @Override + public String getName() { + return "CombineFn: " + combiner.toString(); + } + + @Override + public Combine.CombineFn getCombineFn() { + return combiner; + } + } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java index 539c195bef6d..1c0dae401ee6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -18,40 +18,33 @@ package com.dataartisans.flink.dataflow.translation.wrappers; import com.google.cloud.dataflow.sdk.transforms.Aggregator; -import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; -import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.flink.api.common.accumulators.Accumulator; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.io.Serializable; /** - * Wrapper that wraps a {@link com.google.cloud.dataflow.sdk.transforms.SerializableFunction} + * Wrapper that wraps a {@link com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn} * in a Flink {@link org.apache.flink.api.common.accumulators.Accumulator} for using * the function as an aggregator in a {@link com.google.cloud.dataflow.sdk.transforms.ParDo} * operation. */ -public class SerializableFnAggregatorWrapper implements Aggregator, Accumulator { +public class SerializableFnAggregatorWrapper implements Aggregator, Accumulator { private AO aa; - private SerializableFunction, AO> serFun; + private Combine.CombineFn combiner; - public SerializableFnAggregatorWrapper() { - } - - public SerializableFnAggregatorWrapper(SerializableFunction, AO> serFun) { - this.serFun = serFun; + public SerializableFnAggregatorWrapper(Combine.CombineFn combiner) { + this.combiner = combiner; resetLocal(); } @Override @SuppressWarnings("unchecked") public void add(AI value) { - this.aa = serFun.apply(ImmutableList.of((AI) aa, value)); + this.aa = combiner.apply(ImmutableList.of((AI) aa, value)); } @Override @@ -61,13 +54,13 @@ public Serializable getLocalValue() { @Override public void resetLocal() { - this.aa = serFun.apply(ImmutableList.of()); + this.aa = combiner.apply(ImmutableList.of()); } @Override @SuppressWarnings("unchecked") public void merge(Accumulator other) { - this.aa = serFun.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue())); + this.aa = combiner.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue())); } @Override @@ -75,12 +68,22 @@ public void addValue(AI value) { add(value); } + @Override + public String getName() { + return "Aggregator :" + combiner.toString(); + } + + @Override + public Combine.CombineFn getCombineFn() { + return combiner; + } + @Override public Accumulator clone() { // copy it by merging - AO resultCopy = serFun.apply(Lists.newArrayList((AI) aa)); + AO resultCopy = combiner.apply(Lists.newArrayList((AI) aa)); SerializableFnAggregatorWrapper result = new - SerializableFnAggregatorWrapper<>(serFun); + SerializableFnAggregatorWrapper<>(combiner); result.aa = resultCopy; return result; From df3a347c2ec07af61047658d8922aea50895beaf Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 8 Oct 2015 19:02:07 +0200 Subject: [PATCH 090/149] update ConsoleIO --- .../main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java index aeac1ea95eec..90fb6351bc67 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java @@ -72,7 +72,7 @@ public Bound named(String name) { @Override public PDone apply(PCollection input) { - return new PDone(); + return PDone.in(input.getPipeline()); } } } From 68465f50e5d8c5ec1632b382756029ec789c8005 Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 14 Oct 2015 15:29:05 +0200 Subject: [PATCH 091/149] remove useless job --- .../flink/dataflow/examples/AnotherJOb.java | 51 ------------------- 1 file changed, 51 deletions(-) delete mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/AnotherJOb.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/AnotherJOb.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/AnotherJOb.java deleted file mode 100644 index 7842605cf6af..000000000000 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/AnotherJOb.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 com.dataartisans.flink.dataflow.examples; - -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.google.cloud.dataflow.examples.WordCount; -import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.ParDo; - -public class AnotherJOb { - public static void main(String[] args) { - - WordCount.Options options = PipelineOptionsFactory.create().as(WordCount.Options.class); - options.setOutput("/tmp/output2.txt"); - options.setInput("/Users/max/hello_world.txt"); - //options.setRunner(DirectPipelineRunner.class); - options.setRunner(FlinkPipelineRunner.class); - - Pipeline p = Pipeline.create(options); - - p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) - .apply(ParDo.of(new DoFn() { - - @Override - public void processElement(ProcessContext c) throws Exception { - c.output(c.element()); - } - })) - .apply(TextIO.Write.named("WriteCounts") - .to(options.getOutput()) - .withNumShards(options.getNumShards())); - - p.run(); - } -} From 3b4ef97b668797c4c7f6d515ccab193a55a083e5 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 15 Oct 2015 15:03:34 +0200 Subject: [PATCH 092/149] fix FlinkPipelineOptions --- .../com/dataartisans/flink/dataflow/FlinkPipelineOptions.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java index fdd4e5930b62..e746f414495b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java @@ -16,6 +16,7 @@ package com.dataartisans.flink.dataflow; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.Default; @@ -39,6 +40,7 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp */ @Description("Jar-Files to send to all workers and put on the classpath. " + "The default value is all files from the classpath.") + @JsonIgnore List getFilesToStage(); void setFilesToStage(List value); From b2c85e627bcec2c73f218b5f953f8f272448bf94 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 15 Oct 2015 15:06:06 +0200 Subject: [PATCH 093/149] fix aggregators --- .../dataflow/translation/functions/FlinkDoFnFunction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java index a1d3b984f553..1687ae02d1e4 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java @@ -89,6 +89,7 @@ private class ProcessContext extends DoFn.ProcessContext { public ProcessContext(DoFn fn, Collector outCollector) { fn.super(); + super.setupDelegateAggregators(); this.outCollector = outCollector; } @@ -159,7 +160,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper(combiner); getRuntimeContext().addAccumulator(name, wrapper); - return null; + return wrapper; } From ad30313d3ec425aaa2138acb4dbbe6c7ed12bfbe Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 15 Oct 2015 15:06:32 +0200 Subject: [PATCH 094/149] fix accumulators --- .../translation/functions/FlinkPartialReduceFunction.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java index 3837ffae9d6a..053808d6fb99 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java @@ -49,12 +49,11 @@ public void combine(Iterable> elements, Collector> out) thro K key = first.getKey(); VI value = first.getValue(); VA accumulator = keyedCombineFn.createAccumulator(key); - // manually add for the first element - keyedCombineFn.addInput(key, accumulator, value); + accumulator = keyedCombineFn.addInput(key, accumulator, value); while(iterator.hasNext()) { value = iterator.next().getValue(); - keyedCombineFn.addInput(key, accumulator, value); + accumulator = keyedCombineFn.addInput(key, accumulator, value); } out.collect(KV.of(key, accumulator)); From 4a7de62ddcb76b6e8cb0e15e9b723613dd3c41f7 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 15 Oct 2015 15:06:56 +0200 Subject: [PATCH 095/149] unify transform input and output getters/setters --- .../FlinkTransformTranslators.java | 36 +++++++++---------- .../translation/TranslationContext.java | 20 ----------- 2 files changed, 18 insertions(+), 38 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 542c35c5123d..f9c227c18759 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -149,7 +149,7 @@ public void translateNode(AvroIO.Read.Bound transform, TranslationContext con String path = transform.getFilepattern(); String name = transform.getName(); // Schema schema = transform.getSchema(); - PValue output = context.getCurrentOutput(); + PValue output = context.getOutput(transform); TypeInformation typeInformation = context.getCurrentInputTypeInfo(); @@ -175,7 +175,7 @@ private static class AvroIOWriteTranslator implements FlinkPipelineTranslator @Override public void translateNode(AvroIO.Write.Bound transform, TranslationContext context) { - DataSet inputDataSet = context.getCurrentInputDataSet(); + DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); String filenamePrefix = transform.getFilenamePrefix(); String filenameSuffix = transform.getFilenameSuffix(); int numShards = transform.getNumShards(); @@ -221,7 +221,7 @@ public void translateNode(TextIO.Read.Bound transform, TranslationContex LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType); LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation); - PValue output = (PValue) context.getCurrentTransform().getOutput(); + PValue output = (PValue) context.getOutput(transform); TypeInformation typeInformation = context.getTypeInfo(output); @@ -236,7 +236,7 @@ private static class TextIOWriteTranslator implements FlinkPipelineTranslator @Override public void translateNode(TextIO.Write.Bound transform, TranslationContext context) { - PValue input = (PValue) context.getCurrentTransform().getInput(); + PValue input = context.getInput(transform); DataSet inputDataSet = context.getInputDataSet(input); String filenamePrefix = transform.getFilenamePrefix(); @@ -262,7 +262,7 @@ public void translateNode(TextIO.Write.Bound transform, TranslationContext co private static class ConsoleIOWriteTranslator implements FlinkPipelineTranslator.TransformTranslator { @Override public void translateNode(ConsoleIO.Write.Bound transform, TranslationContext context) { - PValue input = (PValue) context.getCurrentTransform().getInput(); + PValue input = (PValue) context.getInput(transform); DataSet inputDataSet = context.getInputDataSet(input); inputDataSet.printOnTaskManager(transform.getName()); } @@ -272,7 +272,7 @@ private static class GroupByKeyOnlyTranslator implements FlinkPipelineTran @Override public void translateNode(GroupByKey.GroupByKeyOnly transform, TranslationContext context) { - DataSet> inputDataSet = context.getCurrentInputDataSet(); + DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); TypeInformation>> typeInformation = context.getCurrentInputTypeInfo(); @@ -281,7 +281,7 @@ public void translateNode(GroupByKey.GroupByKeyOnly transform, Translation GroupReduceOperator, KV>> outputDataSet = new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); - context.setOutputDataSet(context.getCurrentOutput(), outputDataSet); + context.setOutputDataSet(context.getOutput(transform), outputDataSet); } } @@ -289,7 +289,7 @@ private static class CombinePerKeyTranslator implements FlinkPipe @Override public void translateNode(Combine.PerKey transform, TranslationContext context) { - DataSet> inputDataSet = context.getCurrentInputDataSet(); + DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); @SuppressWarnings("unchecked") Combine.KeyedCombineFn keyedCombineFn = (Combine.KeyedCombineFn) transform.getFn(); @@ -320,7 +320,7 @@ public void translateNode(Combine.PerKey transform, TranslationContex // Reduce fully to VO GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); - TypeInformation> reduceTypeInfo = context.getTypeInfo(context.getCurrentOutput()); + TypeInformation> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform)); Grouping> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); @@ -328,7 +328,7 @@ public void translateNode(Combine.PerKey transform, TranslationContex GroupReduceOperator, KV> outputDataSet = new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName()); - context.setOutputDataSet(context.getCurrentOutput(), outputDataSet); + context.setOutputDataSet(context.getOutput(transform), outputDataSet); } } @@ -357,18 +357,18 @@ private static class ParDoBoundTranslator implements FlinkPipelineTrans @Override public void translateNode(ParDo.Bound transform, TranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getCurrentInput()); + DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); final DoFn doFn = transform.getFn(); - TypeInformation typeInformation = context.getTypeInfo(context.getCurrentOutput()); + TypeInformation typeInformation = context.getTypeInfo(context.getOutput(transform)); FlinkDoFnFunction doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions()); MapPartitionOperator outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName()); transformSideInputs(transform.getSideInputs(), outputDataSet, context); - context.setOutputDataSet(context.getCurrentOutput(), outputDataSet); + context.setOutputDataSet(context.getOutput(transform), outputDataSet); } } @@ -377,7 +377,7 @@ private static class ParDoBoundMultiTranslator implements FlinkPipeline @Override public void translateNode(ParDo.BoundMulti transform, TranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getCurrentInput()); + DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); final DoFn doFn = transform.getFn(); @@ -437,14 +437,14 @@ public void translateNode(Flatten.FlattenPCollectionList transform, Translati result = result.union(current); } } - context.setOutputDataSet(context.getCurrentOutput(), result); + context.setOutputDataSet(context.getOutput(transform), result); } } private static class CreatePCollectionViewTranslator implements FlinkPipelineTranslator.TransformTranslator> { @Override public void translateNode(View.CreatePCollectionView transform, TranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getCurrentInput()); + DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); PCollectionView input = transform.apply(null); context.setSideInputDataSet(input, inputDataSet); } @@ -477,7 +477,7 @@ public void translateNode(Create.Values transform, TranslationContext conte FlinkCreateFunction flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder); FlatMapOperator outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName()); - context.setOutputDataSet(context.getCurrentOutput(), outputDataSet); + context.setOutputDataSet(context.getOutput(transform), outputDataSet); } } @@ -527,7 +527,7 @@ public void translateNode(CoGroupByKey transform, TranslationContext context) DataSet> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2, keySelector1, keySelector2, aggregator, typeInfo, null, transform.getName()); - context.setOutputDataSet(context.getCurrentOutput(), out); + context.setOutputDataSet(context.getOutput(transform), out); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index 7f2d8191ae96..1cfe44566ba5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -73,24 +73,12 @@ public DataSet getInputDataSet(PValue value) { return (DataSet) dataSets.get(value); } - public DataSet getCurrentInputDataSet() { - return getInputDataSet(getCurrentInput()); - } - public void setOutputDataSet(PValue value, DataSet set) { if (!dataSets.containsKey(value)) { dataSets.put(value, set); } } - /** - * Gets the applied AppliedPTransform which carries input/output. - * @return - */ - public AppliedPTransform getCurrentTransform() { - return currentTransform; - } - /** * Sets the AppliedPTransform which carries input/output. * @param currentTransform @@ -131,14 +119,6 @@ public TypeInformation getCurrentOutputTypeInfo() { return getTypeInfo((PValue) currentTransform.getOutput()); } - public PValue getCurrentInput() { - return (PValue) currentTransform.getInput(); - } - - public PValue getCurrentOutput() { - return (PValue) currentTransform.getOutput(); - } - @SuppressWarnings("unchecked") I getInput(PTransform transform) { I input = (I) currentTransform.getInput(); From cdae3d03422cb5b3345971780f4f551e3972205b Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 15 Oct 2015 16:28:58 +0200 Subject: [PATCH 096/149] re-enable and fix Read.Bounded --- .../flink/dataflow/FlinkPipelineRunner.java | 3 ++ .../FlinkTransformTranslators.java | 36 +++++++++-------- .../wrappers/SourceInputFormat.java | 16 ++++---- .../flink/dataflow/ReadSourceITCase.java | 40 +++++++++++-------- 4 files changed, 53 insertions(+), 42 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java index 3636783081ef..6208e728fc5e 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java @@ -120,6 +120,9 @@ private FlinkPipelineRunner(FlinkPipelineOptions options) { this.options = options; this.flinkEnv = createExecutionEnvironment(options); + // set parallelism in the options (required by some execution code) + options.setParallelism(flinkEnv.getParallelism()); + this.translator = new FlinkPipelineTranslator(flinkEnv, options); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index f9c227c18759..3519a7061463 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -27,6 +27,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.io.AvroIO; +import com.google.cloud.dataflow.sdk.io.BoundedSource; import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.Source; import com.google.cloud.dataflow.sdk.io.TextIO; @@ -86,6 +87,7 @@ public class FlinkTransformTranslators { // register the known translators static { TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslator()); + TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator()); // we don't need this because we translate the Combine.PerKey directly // TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator()); @@ -107,7 +109,7 @@ public class FlinkTransformTranslators { //TRANSLATORS.put(PubsubIO.Read.Bound.class, null); //TRANSLATORS.put(PubsubIO.Write.Bound.class, null); -// TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslator()); + TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslator()); // TRANSLATORS.put(Write.Bound.class, new ReadSourceTranslator()); TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslator()); @@ -124,22 +126,22 @@ public static FlinkPipelineTranslator.TransformTranslator getTranslator(PTran return TRANSLATORS.get(transform.getClass()); } -// private static class ReadSourceTranslator implements FlinkPipelineTranslator.TransformTranslator> { -// -// @Override -// public void translateNode(Read.Bounded transform, TranslationContext context) { -// String name = transform.getName(); -// Source source = transform.getSource(); -// PValue output = context.getCurrentTransform().getOutput(). -// Coder coder = output.getCoder(); -// -// TypeInformation typeInformation = context.getTypeInfo(output); -// -// DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), new SourceInputFormat<>(source, context.getPipelineOptions(), coder), typeInformation, name); -// -// context.setOutputDataSet(output, dataSource); -// } -// } + private static class ReadSourceTranslator implements FlinkPipelineTranslator.TransformTranslator> { + + @Override + public void translateNode(Read.Bounded transform, TranslationContext context) { + String name = transform.getName(); + BoundedSource source = transform.getSource(); + PCollection output = context.getOutput(transform); + Coder coder = output.getCoder(); + + TypeInformation typeInformation = context.getTypeInfo(output); + + DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), new SourceInputFormat<>(source, context.getPipelineOptions(), coder), typeInformation, name); + + context.setOutputDataSet(output, dataSource); + } + } private static class AvroIOReadTranslator implements FlinkPipelineTranslator.TransformTranslator> { private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslator.class); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java index 955dceef0dbc..eb1ca4b19f7e 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java @@ -17,9 +17,12 @@ */ package com.dataartisans.flink.dataflow.translation.wrappers; +import com.dataartisans.flink.dataflow.io.ConsoleIO; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.util.Lists; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.BoundedSource; +import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.Source; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.util.WindowedValue; @@ -43,14 +46,14 @@ public class SourceInputFormat implements InputFormat> { private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class); - private final Source initialSource; + private final BoundedSource initialSource; private transient PipelineOptions options; private final Coder coder; - private Source.Reader> reader = null; + private BoundedSource.BoundedReader reader = null; private boolean reachedEnd = true; - public SourceInputFormat(Source initialSource, PipelineOptions options, Coder coder) { + public SourceInputFormat(BoundedSource initialSource, PipelineOptions options, Coder coder) { this.initialSource = initialSource; this.options = options; this.coder = coder; @@ -75,10 +78,7 @@ public void configure(Configuration configuration) {} @Override public void open(SourceInputSplit sourceInputSplit) throws IOException { - WindowedValue.ValueOnlyWindowedValueCoder windowedCoder = WindowedValue - .ValueOnlyWindowedValueCoder.of(coder); - - reader = sourceInputSplit.getSource().createWindowedReader(options, windowedCoder, null); + reader = ((BoundedSource) sourceInputSplit.getSource()).createReader(options); reachedEnd = false; } @@ -157,7 +157,7 @@ public T nextRecord(T t) throws IOException { reachedEnd = !reader.advance(); if (!reachedEnd) { - return reader.getCurrent().getValue(); + return reader.getCurrent(); } return null; } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java index fb68194b55b1..37d9221cca65 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java @@ -18,19 +18,18 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.io.ReadSource; -import com.google.cloud.dataflow.sdk.io.Source; +import com.google.cloud.dataflow.sdk.io.BoundedSource; +import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.cloud.dataflow.sdk.util.ExecutionContext; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.base.Joiner; +import org.apache.flink.shaded.com.google.common.base.Preconditions; import org.apache.flink.test.util.JavaProgramTestBase; -import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -66,7 +65,7 @@ private static void runProgram(String resultPath) { Pipeline p = FlinkTestPipeline.create(); PCollection result = p - .apply(ReadSource.from(new Read(1, 10))) + .apply(Read.from(new ReadSource(1, 10))) .apply(ParDo.of(new DoFn() { @Override public void processElement(ProcessContext c) throws Exception { @@ -79,23 +78,26 @@ public void processElement(ProcessContext c) throws Exception { } } -class Read extends Source { +class ReadSource extends BoundedSource { final int from; final int to; - Read(int from, int to) { + ReadSource(int from, int to) { this.from = from; this.to = to; } @Override - public List splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options) + public List splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options) throws Exception { - List res = new ArrayList<>(); - DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); - float step = 1.0f * (to - from) / dataflowOptions.getNumWorkers(); - for (int i = 0; i < dataflowOptions.getNumWorkers(); ++i) { - res.add(new Read(Math.round(from + i * step), Math.round(from + (i + 1) * step))); + List res = new ArrayList<>(); + FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class); + int numWorkers = flinkOptions.getParallelism(); + Preconditions.checkArgument(numWorkers > 0, "Number of workers should be larger than 0."); + + float step = 1.0f * (to - from) / numWorkers; + for (int i = 0; i < numWorkers; ++i) { + res.add(new ReadSource(Math.round(from + i * step), Math.round(from + (i + 1) * step))); } return res; } @@ -111,8 +113,7 @@ public boolean producesSortedKeys(PipelineOptions options) throws Exception { } @Override - public Reader createBasicReader(PipelineOptions options, Coder coder, - @Nullable ExecutionContext executionContext) throws IOException { + public BoundedReader createReader(PipelineOptions options) throws IOException { return new RangeReader(this); } @@ -124,10 +125,10 @@ public Coder getDefaultOutputCoder() { return BigEndianIntegerCoder.of(); } - private class RangeReader implements Reader { + private class RangeReader extends BoundedReader { private int current; - public RangeReader(Read source) { + public RangeReader(ReadSource source) { this.current = source.from - 1; } @@ -151,6 +152,11 @@ public Integer getCurrent() { public void close() throws IOException { // Nothing } + + @Override + public BoundedSource getCurrentSource() { + return ReadSource.this; + } } } From 758da403fc87ac0e6313a9dd2cfde6d427dd0166 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 15 Oct 2015 16:46:06 +0200 Subject: [PATCH 097/149] fix AvroITCase --- .../dataflow/translation/FlinkTransformTranslators.java | 8 ++++---- .../flink/dataflow/translation/TranslationContext.java | 8 ++++---- .../java/com/dataartisans/flink/dataflow/AvroITCase.java | 3 +-- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 3519a7061463..2d41cd2b4e4a 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -153,7 +153,7 @@ public void translateNode(AvroIO.Read.Bound transform, TranslationContext con // Schema schema = transform.getSchema(); PValue output = context.getOutput(transform); - TypeInformation typeInformation = context.getCurrentInputTypeInfo(); + TypeInformation typeInformation = context.getInputTypeInfo(); // This is super hacky, but unfortunately we cannot get the type otherwise Class avroType = null; @@ -277,7 +277,7 @@ public void translateNode(GroupByKey.GroupByKeyOnly transform, Translation DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); - TypeInformation>> typeInformation = context.getCurrentInputTypeInfo(); + TypeInformation>> typeInformation = context.getInputTypeInfo(); Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); @@ -456,7 +456,7 @@ private static class CreateTranslator implements FlinkPipelineTranslator.Tr @Override public void translateNode(Create.Values transform, TranslationContext context) { - TypeInformation typeInformation = context.getCurrentOutputTypeInfo(); + TypeInformation typeInformation = context.getOutputTypeInfo(); Iterable elements = transform.getElements(); // we need to serialize the elements to byte arrays, since they might contain @@ -519,7 +519,7 @@ public void translateNode(CoGroupByKey transform, TranslationContext context) DataSet> inputDataSet1 = context.getInputDataSet(collection1); DataSet> inputDataSet2 = context.getInputDataSet(collection2); - TypeInformation> typeInfo = context.getCurrentOutputTypeInfo(); + TypeInformation> typeInfo = context.getOutputTypeInfo(); FlinkCoGroupKeyedListAggregator aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index 1cfe44566ba5..7f53e2c41535 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -99,7 +99,7 @@ public void setSideInputDataSet(PCollectionView value, DataSet set) { } @SuppressWarnings("unchecked") - public TypeInformation getTypeInfo(PValue output) { + public TypeInformation getTypeInfo(PInput output) { if (output instanceof TypedPValue) { Coder outputCoder = ((TypedPValue) output).getCoder(); if (outputCoder instanceof KvCoder) { @@ -111,11 +111,11 @@ public TypeInformation getTypeInfo(PValue output) { return new GenericTypeInfo((Class)Object.class); } - public TypeInformation getCurrentInputTypeInfo() { - return getTypeInfo((PValue) currentTransform.getInput()); + public TypeInformation getInputTypeInfo() { + return getTypeInfo(currentTransform.getInput()); } - public TypeInformation getCurrentOutputTypeInfo() { + public TypeInformation getOutputTypeInfo() { return getTypeInfo((PValue) currentTransform.getOutput()); } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java index e9efff239a0b..c6e3e99a6e93 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java @@ -58,8 +58,7 @@ protected void testProgram() throws Exception { private static void runProgram(String tmpPath, String resultPath) { Pipeline p = FlinkTestPipeline.create(); - p.apply(Create.of(new User("Joe", 3, "red"), new User("Mary", 4, "blue"))) - .setCoder(AvroCoder.of(User.class)) + p.apply(Create.of(new User("Joe", 3, "red"), new User("Mary", 4, "blue")).withCoder(AvroCoder.of(User.class))) .apply(AvroIO.Write.to(tmpPath).withSchema(User.class)); p.run(); From add5f3e4c5ea424714c4ef05938d741298e6a5b8 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 15 Oct 2015 16:47:08 +0200 Subject: [PATCH 098/149] re-enable TfIdfITCase --- .../test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java index 457c70366e30..e801ac434540 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java @@ -15,7 +15,7 @@ */ package com.dataartisans.flink.dataflow; -import com.google.cloud.dataflow.examples.TfIdf; +import com.google.cloud.dataflow.examples.complete.TfIdf; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; import com.google.cloud.dataflow.sdk.io.TextIO; From b43dc70ed136a1f2d790b1bad9663337ce5393b5 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 13:38:24 +0200 Subject: [PATCH 099/149] update UnionCoder --- .../translation/functions/UnionCoder.java | 46 ++++++++++--------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java index da5e6dc6c3a3..cd7e04dcb8ba 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java @@ -1,32 +1,34 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * Copyright (C) 2015 Google Inc. * + * Licensed 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. + * 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 com.dataartisans.flink.dataflow.translation.functions; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; + +import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; + import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.coders.StandardCoder; -import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.VarInt; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -75,7 +77,7 @@ public void encode( RawUnionValue union, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException, CoderException { int index = getIndexForEncoding(union); // Write out the union tag. VarInt.encode(index, outStream); @@ -101,13 +103,6 @@ public List> getCoderArguments() { return null; } - @Override - public void verifyDeterministic() throws NonDeterministicException { - for (Coder coder : elementCoders) { - coder.verifyDeterministic(); - } - } - @Override public List> getComponents() { return elementCoders; @@ -148,4 +143,11 @@ public void registerByteSizeObserver( private UnionCoder(List> elementCoders) { this.elementCoders = elementCoders; } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic( + "UnionCoder is only deterministic if all element coders are", + elementCoders); + } } From 9ff4bab3bfdb7dd5ae2e8d7893506b465e28bc58 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 14:45:08 +0200 Subject: [PATCH 100/149] properly translate nested composite transforms --- .../translation/FlinkPipelineTranslator.java | 30 +++++++++++-------- 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java index 855563de3943..885a7d0f914b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java @@ -22,11 +22,9 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; +import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; -import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; -import com.google.cloud.dataflow.sdk.values.PInput; -import com.google.cloud.dataflow.sdk.values.POutput; import com.google.cloud.dataflow.sdk.values.PValue; import org.apache.flink.api.java.ExecutionEnvironment; @@ -41,7 +39,10 @@ public class FlinkPipelineTranslator implements PipelineVisitor { private int depth = 0; - private boolean inComposite = false; + /** + * Composite transform that we want to translate before proceeding with other transforms + */ + private PTransform currentCompositeTransform; public FlinkPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) { this.context = new TranslationContext(env, options); @@ -74,15 +75,15 @@ public void enterCompositeTransform(TransformTreeNode node) { System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); PTransform transform = node.getTransform(); - if (transform != null) { + if (transform != null && currentCompositeTransform == null) { TransformTranslator translator = FlinkTransformTranslators.getTranslator(transform); if (translator != null) { - inComposite = true; + currentCompositeTransform = transform; if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) { // we can only optimize CoGroupByKey for input size 2 - inComposite = false; + currentCompositeTransform = null; } } } @@ -97,10 +98,15 @@ public void leaveCompositeTransform(TransformTreeNode node) { if (transform != null) { TransformTranslator translator = FlinkTransformTranslators.getTranslator(transform); - if (translator != null) { - System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); - applyTransform(transform, node, translator); - inComposite = false; + if (currentCompositeTransform == transform) { + if (translator != null) { + System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); + applyTransform(transform, node, translator); + currentCompositeTransform = null; + } else { + throw new IllegalStateException("Attempted to translate composite transform " + + "but no translator was found: " + currentCompositeTransform); + } } } @@ -111,7 +117,7 @@ public void leaveCompositeTransform(TransformTreeNode node) { @Override public void visitTransform(TransformTreeNode node) { System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node)); - if (inComposite) { + if (currentCompositeTransform != null) { // ignore it return; } From 4aa408126d143606af48b9a74a5e6e105a2171c1 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 14:46:10 +0200 Subject: [PATCH 101/149] translate composite transform GroupByKey --- .../FlinkTransformTranslators.java | 33 ++++++++++++++++--- 1 file changed, 28 insertions(+), 5 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 2d41cd2b4e4a..5a1860e95a72 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -29,7 +29,6 @@ import com.google.cloud.dataflow.sdk.io.AvroIO; import com.google.cloud.dataflow.sdk.io.BoundedSource; import com.google.cloud.dataflow.sdk.io.Read; -import com.google.cloud.dataflow.sdk.io.Source; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.Create; @@ -47,11 +46,9 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionView; -import com.google.cloud.dataflow.sdk.values.POutput; import com.google.cloud.dataflow.sdk.values.PValue; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.Lists; -import org.apache.avro.Schema; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; @@ -90,11 +87,16 @@ public class FlinkTransformTranslators { TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator()); // we don't need this because we translate the Combine.PerKey directly - // TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator()); + //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator()); TRANSLATORS.put(Create.Values.class, new CreateTranslator()); + TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); + TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslator()); + // TODO we're currently ignoring windows here but that has to change in the future + TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator()); + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator()); TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslator()); @@ -277,12 +279,33 @@ public void translateNode(GroupByKey.GroupByKeyOnly transform, Translation DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); - TypeInformation>> typeInformation = context.getInputTypeInfo(); + TypeInformation>> typeInformation = context.getTypeInfo(context.getOutput(transform)); + + Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); + + GroupReduceOperator, KV>> outputDataSet = + new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + } + } + + /** + * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslator} + */ + private static class GroupByKeyTranslator implements FlinkPipelineTranslator.TransformTranslator> { + + @Override + public void translateNode(GroupByKey transform, TranslationContext context) { + DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); + GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); + + TypeInformation>> typeInformation = context.getTypeInfo(context.getOutput(transform)); Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); GroupReduceOperator, KV>> outputDataSet = new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); + context.setOutputDataSet(context.getOutput(transform), outputDataSet); } } From d1d7407055dd7a028d677a430bea1bf717ff329c Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 14:49:46 +0200 Subject: [PATCH 102/149] fix WordCount IT cases --- .../flink/dataflow/WordCountITCase.java | 2 ++ .../flink/dataflow/WordCountJoin2ITCase.java | 17 ++++------------- .../flink/dataflow/WordCountJoin3ITCase.java | 16 ++++------------ 3 files changed, 10 insertions(+), 25 deletions(-) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java index 7bfa0ba8def6..9427ab681196 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java @@ -20,6 +20,7 @@ import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.base.Joiner; import org.apache.flink.test.util.JavaProgramTestBase; @@ -63,6 +64,7 @@ protected void testProgram() throws Exception { input .apply(new WordCount.CountWords()) + .apply(ParDo.of(new WordCount.FormatAsTextFn())) .apply(TextIO.Write.to(resultPath)); p.run(); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java index caddc289ff1c..e1019c53192a 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java @@ -15,10 +15,8 @@ */ package com.dataartisans.flink.dataflow; -import com.google.cloud.dataflow.examples.WordCount; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.*; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; @@ -39,7 +37,7 @@ public class WordCountJoin2ITCase extends JavaProgramTestBase { static final String[] WORDS_2 = new String[] { "hi tim", "beauty", "hooray sue bob", "hi there", "", "please say hi"}; - + static final String[] RESULTS = new String[] { "beauty -> Tag1: Tag2: 1", "bob -> Tag1: 2 Tag2: 1", @@ -69,12 +67,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - WordCount.Options options = PipelineOptionsFactory.create().as(WordCount.Options.class); - options.setOutput(resultPath); - - options.setRunner(FlinkPipelineRunner.class); - - Pipeline p = Pipeline.create(options); + Pipeline p = FlinkTestPipeline.create(); /* Create two PCollections and join them */ PCollection> occurences1 = p.apply(Create.of(WORDS_1)) @@ -93,9 +86,7 @@ protected void testProgram() throws Exception { /* Format output */ mergedOccurences.apply(ParDo.of(new FormatCountsFn())) - .apply(TextIO.Write.named("test") - .to(options.getOutput()) - .withNumShards(options.getNumShards())); + .apply(TextIO.Write.named("test").to(resultPath)); p.run(); } @@ -137,6 +128,6 @@ public void processElement(ProcessContext c) { c.output(key + " -> " + countTag1 + countTag2); } } - + } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java index 585b84c98127..fcdf107c1389 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java @@ -15,10 +15,8 @@ */ package com.dataartisans.flink.dataflow; -import com.google.cloud.dataflow.examples.WordCount; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.*; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; @@ -43,7 +41,7 @@ public class WordCountJoin3ITCase extends JavaProgramTestBase { static final String[] WORDS_3 = new String[] { "hi stephan", "beauty", "hooray big fabian", "hi yo", "", "please say hi"}; - + static final String[] RESULTS = new String[] { "beauty -> Tag1: Tag2: 1 Tag3: 1", "bob -> Tag1: 2 Tag2: 1 Tag3: ", @@ -78,12 +76,8 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - WordCount.Options options = PipelineOptionsFactory.create().as(WordCount.Options.class); - options.setOutput(resultPath); - - options.setRunner(FlinkPipelineRunner.class); - Pipeline p = Pipeline.create(options); + Pipeline p = FlinkTestPipeline.create(); /* Create two PCollections and join them */ PCollection> occurences1 = p.apply(Create.of(WORDS_1)) @@ -107,9 +101,7 @@ protected void testProgram() throws Exception { /* Format output */ mergedOccurences.apply(ParDo.of(new FormatCountsFn())) - .apply(TextIO.Write.named("test") - .to(options.getOutput()) - .withNumShards(options.getNumShards())); + .apply(TextIO.Write.named("test").to(resultPath)); p.run(); } @@ -155,5 +147,5 @@ public void processElement(ProcessContext c) { c.output(key + " -> " + countTag1 + countTag2 + countTag3); } } - + } From 601d1b69c2d980e9fde52d10a89b1901389daec1 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 14:50:38 +0200 Subject: [PATCH 103/149] fix WordCount example program --- .../flink/dataflow/examples/WordCount.java | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java index 6fd04891ae6e..6b44f5a77b5a 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.options.Description; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.ParDo; public class WordCount { @@ -41,28 +42,22 @@ public static interface Options extends PipelineOptions, FlinkPipelineOptions { @Description("Path of the file to write to") String getOutput(); void setOutput(String value); - - /** - * By default (numShards == 0), the system will choose the shard count. - * Most programs will not need this option. - */ - @Description("Number of output shards (0 if the system should choose automatically)") - int getNumShards(); - void setNumShards(int value); } public static void main(String[] args) { - Options options = PipelineOptionsFactory.fromArgs(args).as(Options.class); + Options options = PipelineOptionsFactory.fromArgs(args).withValidation() + .as(Options.class); options.setRunner(FlinkPipelineRunner.class); Pipeline p = Pipeline.create(options); + // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the + // static FormatAsTextFn() to the ParDo transform. p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) .apply(new CountWords()) - .apply(TextIO.Write.named("WriteCounts") - .to(options.getOutput()) - .withNumShards(options.getNumShards())); + .apply(ParDo.of(new com.google.cloud.dataflow.examples.WordCount.FormatAsTextFn())) + .apply(TextIO.Write.named("WriteCounts").to(options.getOutput())); p.run(); } From 58749901ad627cff67d72897ef538d72be86be6d Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 14:52:50 +0200 Subject: [PATCH 104/149] add missing interface methods for DoFn wrapper --- .../functions/FlinkDoFnFunction.java | 39 ++++++++++++++++++- .../FlinkKeyedListAggregationFunction.java | 2 - 2 files changed, 38 insertions(+), 3 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java index 1687ae02d1e4..02af91d62cdc 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java @@ -18,7 +18,10 @@ import com.dataartisans.flink.dataflow.translation.wrappers.CombineFnAggregatorWrapper; import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.BooleanNode; +import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.repackaged.com.google.common.collect.Lists; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.DoFn; @@ -26,8 +29,10 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; +import com.google.cloud.dataflow.sdk.util.TimerInternals; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowingInternals; +import com.google.cloud.dataflow.sdk.util.state.StateInternals; import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableList; @@ -41,6 +46,8 @@ import java.io.ObjectOutputStream; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; +import java.util.LinkedList; import java.util.List; /** @@ -116,7 +123,37 @@ public PaneInfo pane() { @Override public WindowingInternals windowingInternals() { - return null; + return new WindowingInternals() { + @Override + public StateInternals stateInternals() { + return null; + } + + @Override + public void outputWindowedValue(OUT output, Instant timestamp, Collection windows, PaneInfo pane) { + + } + + @Override + public TimerInternals timerInternals() { + return null; + } + + @Override + public Collection windows() { + return ImmutableList.of(GlobalWindow.INSTANCE); + } + + @Override + public PaneInfo pane() { + return PaneInfo.NO_FIRING; + } + + @Override + public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { + + } + }; } @Override diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java index 48da66cb7d9d..5d3702a483c6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java @@ -19,9 +19,7 @@ import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.util.Collector; -import java.util.ArrayList; import java.util.Iterator; -import java.util.List; /** * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a From 2a3612dc17efdb1d361fd3697e8ba0dcbb38bd73 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 15:25:54 +0200 Subject: [PATCH 105/149] fix TfIdf example --- .../flink/dataflow/examples/TFIDF.java | 427 +++++++++++++++--- 1 file changed, 354 insertions(+), 73 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java index e425c1688424..60146c9ef357 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java @@ -1,46 +1,101 @@ /* - * Copyright 2015 Data Artisans GmbH + * Copyright (C) 2015 Google Inc. * - * Licensed 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 + * Licensed 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 + * 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. + * 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 com.dataartisans.flink.dataflow.examples; +import com.dataartisans.flink.dataflow.FlinkPipelineOptions; import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.google.cloud.dataflow.examples.TfIdf; import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.*; -import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.GcsOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation; +import com.google.cloud.dataflow.sdk.transforms.Count; import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.Flatten; +import com.google.cloud.dataflow.sdk.transforms.Keys; +import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; +import com.google.cloud.dataflow.sdk.transforms.Values; +import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.transforms.WithKeys; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; import com.google.cloud.dataflow.sdk.util.GcsUtil; import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; -import com.google.common.collect.Iterables; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.PDone; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.TupleTag; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; -import java.util.ArrayList; import java.util.HashSet; -import java.util.List; import java.util.Set; +/** + * An example that computes a basic TF-IDF search table for a directory or GCS prefix. + * + *

Concepts: joining data; side inputs; logging + * + *

To execute this pipeline locally, specify general pipeline configuration: + *

{@code
+ *   --project=YOUR_PROJECT_ID
+ * }
+ * and a local output file or output prefix on GCS: + *
{@code
+ *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
+ * }
+ * + *

To execute this pipeline using the Dataflow service, specify pipeline configuration: + *

{@code
+ *   --project=YOUR_PROJECT_ID
+ *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --runner=BlockingDataflowPipelineRunner
+ * and an output prefix on GCS:
+ *   --output=gs://YOUR_OUTPUT_PREFIX
+ * }
+ * + *

The default input is {@code gs://dataflow-samples/shakespeare/} and can be overridden with + * {@code --input}. + */ public class TFIDF { - - private static interface Options extends PipelineOptions { + /** + * Options supported by {@link TFIDF}. + *

+ * Inherits standard configuration options. + */ + private static interface Options extends PipelineOptions, FlinkPipelineOptions { @Description("Path to the directory or GCS prefix containing files to read from") @Default.String("gs://dataflow-samples/shakespeare/") String getInput(); @@ -52,6 +107,9 @@ private static interface Options extends PipelineOptions { void setOutput(String value); } + /** + * Lists documents contained beneath the {@code options.input} prefix/directory. + */ public static Set listInputDocuments(Options options) throws URISyntaxException, IOException { URI baseUri = new URI(options.getInput()); @@ -92,82 +150,305 @@ public static Set listInputDocuments(Options options) return uris; } - static class SimpleCombineFn extends Combine.CombineFn, V> { - /** - * Returns a {@code CombineFn} that uses the given - * {@code SerializableFunction} to combine values. - */ - public static SimpleCombineFn of( - SerializableFunction, V> combiner) { - return new SimpleCombineFn<>(combiner); - } - - /** - * The number of values to accumulate before invoking the combiner - * function to combine them. - */ - private static final int BUFFER_SIZE = 20; + /** + * Reads the documents at the provided uris and returns all lines + * from the documents tagged with which document they are from. + */ + public static class ReadDocuments + extends PTransform>> { + private static final long serialVersionUID = 0; - /** The combiner function. */ - private final SerializableFunction, V> combiner; + private Iterable uris; - private SimpleCombineFn(SerializableFunction, V> combiner) { - this.combiner = combiner; + public ReadDocuments(Iterable uris) { + this.uris = uris; } @Override - public List createAccumulator() { - return new ArrayList<>(); + public Coder getDefaultOutputCoder() { + return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of()); } @Override - public void addInput(List accumulator, V input) { - accumulator.add(input); - if (accumulator.size() > BUFFER_SIZE) { - V combined = combiner.apply(accumulator); - accumulator.clear(); - accumulator.add(combined); + public PCollection> apply(PInput input) { + Pipeline pipeline = input.getPipeline(); + + // Create one TextIO.Read transform for each document + // and add its output to a PCollectionList + PCollectionList> urisToLines = + PCollectionList.empty(pipeline); + + // TextIO.Read supports: + // - file: URIs and paths locally + // - gs: URIs on the service + for (final URI uri : uris) { + String uriString; + if (uri.getScheme().equals("file")) { + uriString = new File(uri).getPath(); + } else { + uriString = uri.toString(); + } + + PCollection> oneUriToLines = pipeline + .apply(TextIO.Read.from(uriString) + .named("TextIO.Read(" + uriString + ")")) + .apply("WithKeys(" + uriString + ")", WithKeys.of(uri)); + + urisToLines = urisToLines.and(oneUriToLines); } - } - @Override - public List mergeAccumulators(Iterable> accumulators) { - List singleton = new ArrayList<>(); - singleton.add(combiner.apply(Iterables.concat(accumulators))); - return singleton; + return urisToLines.apply(Flatten.>pCollections()); } + } + + /** + * A transform containing a basic TF-IDF pipeline. The input consists of KV objects + * where the key is the document's URI and the value is a piece + * of the document's content. The output is mapping from terms to + * scores for each document URI. + */ + public static class ComputeTfIdf + extends PTransform>, PCollection>>> { + private static final long serialVersionUID = 0; + + public ComputeTfIdf() { } @Override - public V extractOutput(List accumulator) { - return combiner.apply(accumulator); + public PCollection>> apply( + PCollection> uriToContent) { + + // Compute the total number of documents, and + // prepare this singleton PCollectionView for + // use as a side input. + final PCollectionView totalDocuments = + uriToContent + .apply("GetURIs", Keys.create()) + .apply("RemoveDuplicateDocs", RemoveDuplicates.create()) + .apply(Count.globally()) + .apply(View.asSingleton()); + + // Create a collection of pairs mapping a URI to each + // of the words in the document associated with that that URI. + PCollection> uriToWords = uriToContent + .apply(ParDo.named("SplitWords").of( + new DoFn, KV>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey(); + String line = c.element().getValue(); + for (String word : line.split("\\W+")) { + // Log INFO messages when the word “love” is found. + if (word.toLowerCase().equals("love")) { + LOG.info("Found {}", word.toLowerCase()); + } + + if (!word.isEmpty()) { + c.output(KV.of(uri, word.toLowerCase())); + } + } + } + })); + + // Compute a mapping from each word to the total + // number of documents in which it appears. + PCollection> wordToDocCount = uriToWords + .apply("RemoveDuplicateWords", RemoveDuplicates.>create()) + .apply(Values.create()) + .apply("CountDocs", Count.perElement()); + + // Compute a mapping from each URI to the total + // number of words in the document associated with that URI. + PCollection> uriToWordTotal = uriToWords + .apply("GetURIs2", Keys.create()) + .apply("CountWords", Count.perElement()); + + // Count, for each (URI, word) pair, the number of + // occurrences of that word in the document associated + // with the URI. + PCollection, Long>> uriAndWordToCount = uriToWords + .apply("CountWordDocPairs", Count.>perElement()); + + // Adjust the above collection to a mapping from + // (URI, word) pairs to counts into an isomorphic mapping + // from URI to (word, count) pairs, to prepare for a join + // by the URI key. + PCollection>> uriToWordAndCount = uriAndWordToCount + .apply(ParDo.named("ShiftKeys").of( + new DoFn, Long>, KV>>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey().getKey(); + String word = c.element().getKey().getValue(); + Long occurrences = c.element().getValue(); + c.output(KV.of(uri, KV.of(word, occurrences))); + } + })); + + // Prepare to join the mapping of URI to (word, count) pairs with + // the mapping of URI to total word counts, by associating + // each of the input PCollection> with + // a tuple tag. Each input must have the same key type, URI + // in this case. The type parameter of the tuple tag matches + // the types of the values for each collection. + final TupleTag wordTotalsTag = new TupleTag(); + final TupleTag> wordCountsTag = new TupleTag>(); + KeyedPCollectionTuple coGbkInput = KeyedPCollectionTuple + .of(wordTotalsTag, uriToWordTotal) + .and(wordCountsTag, uriToWordAndCount); + + // Perform a CoGroupByKey (a sort of pre-join) on the prepared + // inputs. This yields a mapping from URI to a CoGbkResult + // (CoGroupByKey Result). The CoGbkResult is a mapping + // from the above tuple tags to the values in each input + // associated with a particular URI. In this case, each + // KV group a URI with the total number of + // words in that document as well as all the (word, count) + // pairs for particular words. + PCollection> uriToWordAndCountAndTotal = coGbkInput + .apply("CoGroupByUri", CoGroupByKey.create()); + + // Compute a mapping from each word to a (URI, term frequency) + // pair for each URI. A word's term frequency for a document + // is simply the number of times that word occurs in the document + // divided by the total number of words in the document. + PCollection>> wordToUriAndTf = uriToWordAndCountAndTotal + .apply(ParDo.named("ComputeTermFrequencies").of( + new DoFn, KV>>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey(); + Long wordTotal = c.element().getValue().getOnly(wordTotalsTag); + + for (KV wordAndCount + : c.element().getValue().getAll(wordCountsTag)) { + String word = wordAndCount.getKey(); + Long wordCount = wordAndCount.getValue(); + Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue(); + c.output(KV.of(word, KV.of(uri, termFrequency))); + } + } + })); + + // Compute a mapping from each word to its document frequency. + // A word's document frequency in a corpus is the number of + // documents in which the word appears divided by the total + // number of documents in the corpus. Note how the total number of + // documents is passed as a side input; the same value is + // presented to each invocation of the DoFn. + PCollection> wordToDf = wordToDocCount + .apply(ParDo + .named("ComputeDocFrequencies") + .withSideInputs(totalDocuments) + .of(new DoFn, KV>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + String word = c.element().getKey(); + Long documentCount = c.element().getValue(); + Long documentTotal = c.sideInput(totalDocuments); + Double documentFrequency = documentCount.doubleValue() + / documentTotal.doubleValue(); + + c.output(KV.of(word, documentFrequency)); + } + })); + + // Join the term frequency and document frequency + // collections, each keyed on the word. + final TupleTag> tfTag = new TupleTag>(); + final TupleTag dfTag = new TupleTag(); + PCollection> wordToUriAndTfAndDf = KeyedPCollectionTuple + .of(tfTag, wordToUriAndTf) + .and(dfTag, wordToDf) + .apply(CoGroupByKey.create()); + + // Compute a mapping from each word to a (URI, TF-IDF) score + // for each URI. There are a variety of definitions of TF-IDF + // ("term frequency - inverse document frequency") score; + // here we use a basic version that is the term frequency + // divided by the log of the document frequency. + PCollection>> wordToUriAndTfIdf = wordToUriAndTfAndDf + .apply(ParDo.named("ComputeTfIdf").of( + new DoFn, KV>>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + String word = c.element().getKey(); + Double df = c.element().getValue().getOnly(dfTag); + + for (KV uriAndTf : c.element().getValue().getAll(tfTag)) { + URI uri = uriAndTf.getKey(); + Double tf = uriAndTf.getValue(); + Double tfIdf = tf * Math.log(1 / df); + c.output(KV.of(word, KV.of(uri, tfIdf))); + } + } + })); + + return wordToUriAndTfIdf; } + + // Instantiate Logger. + // It is suggested that the user specify the class name of the containing class + // (in this case ComputeTfIdf). + private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class); } - public static void main(String[] args) throws IOException, URISyntaxException { - Options options = PipelineOptionsFactory.create().as(Options.class); - options.setOutput("/tmp/output2.txt"); - options.setInput("/tmp/documents"); - //options.setRunner(DirectPipelineRunner.class); - options.setRunner(FlinkPipelineRunner.class); + /** + * A {@link PTransform} to write, in CSV format, a mapping from term and URI + * to score. + */ + public static class WriteTfIdf + extends PTransform>>, PDone> { + private static final long serialVersionUID = 0; + + private String output; + + public WriteTfIdf(String output) { + this.output = output; + } + + @Override + public PDone apply(PCollection>> wordToUriAndTfIdf) { + return wordToUriAndTfIdf + .apply(ParDo.named("Format").of(new DoFn>, String>() { + private static final long serialVersionUID = 0; - Pipeline p = Pipeline.create(options); + @Override + public void processElement(ProcessContext c) { + c.output(String.format("%s,\t%s,\t%f", + c.element().getKey(), + c.element().getValue().getKey(), + c.element().getValue().getValue())); + } + })) + .apply(TextIO.Write + .to(output) + .withSuffix(".csv")); + } + } - p.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); + public static void main(String[] args) throws Exception { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - List documentURLs = new ArrayList(); + options.setRunner(FlinkPipelineRunner.class); + Pipeline pipeline = Pipeline.create(options); + pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); - p.apply(new TfIdf.ReadDocuments(listInputDocuments(options))) - .apply(new TfIdf.ComputeTfIdf()) - .apply(ParDo.of(new DoFn() { - @Override - public void processElement(ProcessContext c) throws Exception { - c.output(c.element().toString()); - } - })) - .apply(TextIO.Write.to("/tmp/output")); - //.apply(new TfIdf.WriteTfIdf(options.getOutput())); + pipeline + .apply(new ReadDocuments(listInputDocuments(options))) + .apply(new ComputeTfIdf()) + .apply(new WriteTfIdf(options.getOutput())); - p.run(); + pipeline.run(); } } From 8d9ca54d0886b999bd1c88c0286969771eef5b9e Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 15:48:45 +0200 Subject: [PATCH 106/149] add SideInputITCase --- .../flink/dataflow/SideInputITCase.java} | 56 +++++++++++-------- 1 file changed, 32 insertions(+), 24 deletions(-) rename runners/flink/src/{main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java => test/java/com/dataartisans/flink/dataflow/SideInputITCase.java} (50%) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java similarity index 50% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java rename to runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java index a295223cf658..d932c80ffb71 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/SideInputTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java @@ -13,47 +13,55 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.examples; +package com.dataartisans.flink.dataflow; -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.google.cloud.dataflow.examples.WordCount; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.View; import com.google.cloud.dataflow.sdk.values.PCollectionView; +import org.apache.flink.test.util.JavaProgramTestBase; -import java.io.IOException; -import java.net.URISyntaxException; +import java.io.Serializable; -public class SideInputTest { - public static void main(String[] args) throws IOException, URISyntaxException { - WordCount.Options options = PipelineOptionsFactory.create().as(WordCount.Options.class); - options.setOutput("/tmp/output2.txt"); - options.setInput("/tmp/documents/hello_world.txt"); - //options.setRunner(DirectPipelineRunner.class); - options.setRunner(FlinkPipelineRunner.class); +public class SideInputITCase extends JavaProgramTestBase implements Serializable { - Pipeline p = Pipeline.create(options); + private static final String expected = "Hello!"; - final PCollectionView totalDocuments = p - .apply(Create.of("Hello!")) - .apply(View.asSingleton()); + protected String resultPath; - p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) - .apply(ParDo.of(new DoFn() { + @Override + protected void testProgram() throws Exception { + + + Pipeline p = FlinkTestPipeline.create(); + + final PCollectionView sidesInput = p + .apply(Create.of(expected)) + .apply(View.asSingleton()); + + p.apply(Create.of("bli")) + .apply(ParDo.of(new DoFn() { @Override public void processElement(ProcessContext c) throws Exception { - String s = c.sideInput(totalDocuments); - System.out.println("side Input:" + s); - c.output(c.element()); + String s = c.sideInput(sidesInput); + c.output(s); } - }).withSideInputs(totalDocuments)).apply(TextIO.Write.to("/tmp/output")); - + }).withSideInputs(sidesInput)).apply(TextIO.Write.to(resultPath)); + p.run(); } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expected, resultPath); + } } From edaa34b79dbe3649a93ead179fa2eb8a69776c71 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 15:50:10 +0200 Subject: [PATCH 107/149] add MayBeEmptyITCase --- .../flink/dataflow/MaybeEmptyTestITCase.java} | 53 ++++++++++--------- 1 file changed, 27 insertions(+), 26 deletions(-) rename runners/flink/src/{main/java/com/dataartisans/flink/dataflow/examples/MaybeEmptyTest.java => test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java} (57%) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/MaybeEmptyTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java similarity index 57% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/MaybeEmptyTest.java rename to runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java index 24257c8d1471..d8b392922eaa 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/MaybeEmptyTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.examples; +package com.dataartisans.flink.dataflow; import com.dataartisans.flink.dataflow.FlinkPipelineRunner; import com.google.cloud.dataflow.sdk.Pipeline; @@ -23,43 +23,44 @@ import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.common.base.Joiner; +import org.apache.flink.test.util.JavaProgramTestBase; -/** - * Created by max on 18/02/15. - */ -public class MaybeEmptyTest { +import java.io.Serializable; + +public class MaybeEmptyTestITCase extends JavaProgramTestBase implements Serializable { + + protected String resultPath; - private static interface Options extends PipelineOptions { - @Description("Path to the directory or GCS prefix containing files to read from") - @Default.String("gs://dataflow-samples/shakespeare/") - String getInput(); - void setInput(String value); + protected final String expected = "test"; - @Description("Prefix of output URI to write to") - @Validation.Required - String getOutput(); - void setOutput(String value); + public MaybeEmptyTestITCase() { } - - public static void main(String[] args ){ - Options options = PipelineOptionsFactory.create().as(Options.class); - options.setOutput("/tmp/output2.txt"); - options.setInput("/tmp/documents"); - //options.setRunner(DirectPipelineRunner.class); - options.setRunner(FlinkPipelineRunner.class); - Pipeline p = Pipeline.create(options); + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expected, resultPath); + } + + @Override + protected void testProgram() throws Exception { + + Pipeline p = FlinkTestPipeline.create(); p.apply(Create.of((Void) null)).setCoder(VoidCoder.of()) .apply(ParDo.of( new DoFn() { @Override public void processElement(DoFn.ProcessContext c) { - System.out.println("hello"); - c.output("test"); + c.output(expected); } - })).apply(TextIO.Write.to("bla")); + })).apply(TextIO.Write.to(resultPath)); p.run(); - } + } From dc9df3881beb0fdd6ce8513389edbd65eb425178 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 15:50:41 +0200 Subject: [PATCH 108/149] fix WordCount comment --- .../com/dataartisans/flink/dataflow/examples/WordCount.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java index 6b44f5a77b5a..34ad5c400d40 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java @@ -52,8 +52,6 @@ public static void main(String[] args) { Pipeline p = Pipeline.create(options); - // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the - // static FormatAsTextFn() to the ParDo transform. p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) .apply(new CountWords()) .apply(ParDo.of(new com.google.cloud.dataflow.examples.WordCount.FormatAsTextFn())) From 1d8462af46d1e47945a55489256e1f54e9cd733a Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 15:51:11 +0200 Subject: [PATCH 109/149] use correct output type --- .../flink/dataflow/translation/FlinkTransformTranslators.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 5a1860e95a72..43f32fd90013 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -155,7 +155,7 @@ public void translateNode(AvroIO.Read.Bound transform, TranslationContext con // Schema schema = transform.getSchema(); PValue output = context.getOutput(transform); - TypeInformation typeInformation = context.getInputTypeInfo(); + TypeInformation typeInformation = context.getTypeInfo(output); // This is super hacky, but unfortunately we cannot get the type otherwise Class avroType = null; From 2dbfd89f358cae0eb429a9ffac37f8b3475b30a7 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 15:51:42 +0200 Subject: [PATCH 110/149] disable Wikipedia test (for now) --- .../flink/dataflow/TopWikipediaSessions.java | 400 +++++++++--------- 1 file changed, 210 insertions(+), 190 deletions(-) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java index 07b6adfdeb5b..ab5565a7e220 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java @@ -1,190 +1,210 @@ -/* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 com.dataartisans.flink.dataflow; - -import com.google.api.services.bigquery.model.TableRow; -import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder; -import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; -import com.google.cloud.dataflow.sdk.options.Default; -import com.google.cloud.dataflow.sdk.options.Description; -import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.options.Validation; -import com.google.cloud.dataflow.sdk.transforms.Count; -import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.cloud.dataflow.sdk.transforms.SerializableComparator; -import com.google.cloud.dataflow.sdk.transforms.Top; -import com.google.cloud.dataflow.sdk.transforms.windowing.CalendarWindows; -import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; -import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; -import com.google.cloud.dataflow.sdk.transforms.windowing.Window; -import com.google.cloud.dataflow.sdk.values.KV; -import com.google.cloud.dataflow.sdk.values.PCollection; -import org.joda.time.Duration; -import org.joda.time.Instant; - -import java.util.List; - -/** - * Copied from {@link com.google.cloud.dataflow.examples.JoinExamples} because the code - * is private there. - */ -public class TopWikipediaSessions { - private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json"; - - /** - * Extracts user and timestamp from a TableRow representing a Wikipedia edit. - */ - static class ExtractUserAndTimestamp extends DoFn { - @Override - public void processElement(ProcessContext c) { - TableRow row = c.element(); - int timestamp = (Integer) row.get("timestamp"); - String userName = (String) row.get("contributor_username"); - if (userName != null) { - // Sets the implicit timestamp field to be used in windowing. - c.outputWithTimestamp(userName, new Instant(timestamp * 1000L)); - } - } - } - - /** - * Computes the number of edits in each user session. A session is defined as - * a string of edits where each is separated from the next by less than an hour. - */ - static class ComputeSessions - extends PTransform, PCollection>> { - @Override - public PCollection> apply(PCollection actions) { - return actions - .apply(Window.into(Sessions.withGapDuration(Duration.standardHours(1)))) - - .apply(Count.perElement()); - } - } - - /** - * Computes the longest session ending in each month. - */ - private static class TopPerMonth - extends PTransform>, PCollection>>> { - @Override - public PCollection>> apply(PCollection> sessions) { - return sessions - .apply(Window.>into(CalendarWindows.months(1))) - - .apply(Top.of(1, new SerializableComparator>() { - @Override - public int compare(KV o1, KV o2) { - return Long.compare(o1.getValue(), o2.getValue()); - } - })); - } - } - - static class ComputeTopSessions extends PTransform, PCollection> { - private final double samplingThreshold; - - public ComputeTopSessions(double samplingThreshold) { - this.samplingThreshold = samplingThreshold; - } - - @Override - public PCollection apply(PCollection input) { - return input - .apply(ParDo.of(new ExtractUserAndTimestamp())) - - .apply(ParDo.named("SampleUsers").of( - new DoFn() { - @Override - public void processElement(ProcessContext c) { - if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * samplingThreshold) { - c.output(c.element()); - } - } - })) - - .apply(new ComputeSessions()) - - .apply(ParDo.named("SessionsToStrings").of( - new DoFn, KV>() { - @Override - public void processElement(ProcessContext c) { - c.output(KV.of( - c.element().getKey() + " : " - + c.windows().iterator().next(), c.element().getValue())); - } - })) - - .apply(new TopPerMonth()) - - .apply(ParDo.named("FormatOutput").of( - new DoFn>, String>() { - @Override - public void processElement(ProcessContext c) { - for (KV item : c.element()) { - String session = item.getKey(); - long count = item.getValue(); - c.output( - session + " : " + count + " : " - + ((IntervalWindow) c.windows().iterator().next()).start()); - } - } - })); - } - } - - /** - * Options supported by this class. - * - *

Inherits standard Dataflow configuration options. - */ - private static interface Options extends PipelineOptions { - @Description( - "Input specified as a GCS path containing a BigQuery table exported as json") - @Default.String(EXPORTED_WIKI_TABLE) - String getInput(); - void setInput(String value); - - @Description("File to output results to") - @Validation.Required - String getOutput(); - void setOutput(String value); - } - - public static void main(String[] args) { - Options options = PipelineOptionsFactory.fromArgs(args) - .withValidation() - .as(Options.class); - DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); - - Pipeline p = Pipeline.create(dataflowOptions); - - double samplingThreshold = 0.1; - - p.apply(TextIO.Read - .from(options.getInput()) - .withCoder(TableRowJsonCoder.of())) - .apply(new ComputeTopSessions(samplingThreshold)) - .apply(TextIO.Write.named("Write").withoutSharding().to(options.getOutput())); - - p.run(); - } -} +///* +// * Copyright (C) 2015 Google Inc. +// * +// * Licensed 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 com.dataartisans.flink.dataflow; +// +//import com.google.api.services.bigquery.model.TableRow; +//import com.google.cloud.dataflow.sdk.Pipeline; +//import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder; +//import com.google.cloud.dataflow.sdk.io.TextIO; +//import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +//import com.google.cloud.dataflow.sdk.options.Default; +//import com.google.cloud.dataflow.sdk.options.Description; +//import com.google.cloud.dataflow.sdk.options.PipelineOptions; +//import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +//import com.google.cloud.dataflow.sdk.options.Validation; +//import com.google.cloud.dataflow.sdk.transforms.Count; +//import com.google.cloud.dataflow.sdk.transforms.DoFn; +//import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess; +//import com.google.cloud.dataflow.sdk.transforms.PTransform; +//import com.google.cloud.dataflow.sdk.transforms.ParDo; +//import com.google.cloud.dataflow.sdk.transforms.SerializableComparator; +//import com.google.cloud.dataflow.sdk.transforms.Top; +//import com.google.cloud.dataflow.sdk.transforms.windowing.CalendarWindows; +//import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +//import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +//import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +//import com.google.cloud.dataflow.sdk.values.KV; +//import com.google.cloud.dataflow.sdk.values.PCollection; +// +//import org.joda.time.Duration; +//import org.joda.time.Instant; +// +//import java.util.List; +// +///** +// * Copied from {@link com.google.cloud.dataflow.examples.complete.TopWikipediaSessions} because the code +// * is private there. +// */ +//public class TopWikipediaSessions { +// private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json"; +// +// /** +// * Extracts user and timestamp from a TableRow representing a Wikipedia edit. +// */ +// static class ExtractUserAndTimestamp extends DoFn { +// private static final long serialVersionUID = 0; +// +// @Override +// public void processElement(ProcessContext c) { +// TableRow row = c.element(); +// int timestamp = (Integer) row.get("timestamp"); +// String userName = (String) row.get("contributor_username"); +// if (userName != null) { +// // Sets the implicit timestamp field to be used in windowing. +// c.outputWithTimestamp(userName, new Instant(timestamp * 1000L)); +// } +// } +// } +// +// /** +// * Computes the number of edits in each user session. A session is defined as +// * a string of edits where each is separated from the next by less than an hour. +// */ +// static class ComputeSessions +// extends PTransform, PCollection>> { +// private static final long serialVersionUID = 0; +// +// @Override +// public PCollection> apply(PCollection actions) { +// return actions +// .apply(Window.into(Sessions.withGapDuration(Duration.standardHours(1)))) +// +// .apply(Count.perElement()); +// } +// } +// +// /** +// * Computes the longest session ending in each month. +// */ +// private static class TopPerMonth +// extends PTransform>, PCollection>>> { +// private static final long serialVersionUID = 0; +// +// @Override +// public PCollection>> apply(PCollection> sessions) { +// return sessions +// .apply(Window.>into(CalendarWindows.months(1))) +// +// .apply(Top.of(1, new SerializableComparator>() { +// private static final long serialVersionUID = 0; +// +// @Override +// public int compare(KV o1, KV o2) { +// return Long.compare(o1.getValue(), o2.getValue()); +// } +// }).withoutDefaults()); +// } +// } +// +// static class SessionsToStringsDoFn extends DoFn, KV> +// implements RequiresWindowAccess { +// +// private static final long serialVersionUID = 0; +// +// @Override +// public void processElement(ProcessContext c) { +// c.output(KV.of( +// c.element().getKey() + " : " + c.window(), c.element().getValue())); +// } +// } +// +// static class FormatOutputDoFn extends DoFn>, String> +// implements RequiresWindowAccess { +// private static final long serialVersionUID = 0; +// +// @Override +// public void processElement(ProcessContext c) { +// for (KV item : c.element()) { +// String session = item.getKey(); +// long count = item.getValue(); +// c.output(session + " : " + count + " : " + ((IntervalWindow) c.window()).start()); +// } +// } +// } +// +// static class ComputeTopSessions extends PTransform, PCollection> { +// +// private static final long serialVersionUID = 0; +// +// private final double samplingThreshold; +// +// public ComputeTopSessions(double samplingThreshold) { +// this.samplingThreshold = samplingThreshold; +// } +// +// @Override +// public PCollection apply(PCollection input) { +// return input +// .apply(ParDo.of(new ExtractUserAndTimestamp())) +// +// .apply(ParDo.named("SampleUsers").of( +// new DoFn() { +// private static final long serialVersionUID = 0; +// +// @Override +// public void processElement(ProcessContext c) { +// if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * samplingThreshold) { +// c.output(c.element()); +// } +// } +// })) +// +// .apply(new ComputeSessions()) +// +// .apply(ParDo.named("SessionsToStrings").of(new SessionsToStringsDoFn())) +// .apply(new TopPerMonth()) +// .apply(ParDo.named("FormatOutput").of(new FormatOutputDoFn())); +// } +// } +// +// /** +// * Options supported by this class. +// * +// *

Inherits standard Dataflow configuration options. +// */ +// private static interface Options extends PipelineOptions { +// @Description( +// "Input specified as a GCS path containing a BigQuery table exported as json") +// @Default.String(EXPORTED_WIKI_TABLE) +// String getInput(); +// void setInput(String value); +// +// @Description("File to output results to") +// @Validation.Required +// String getOutput(); +// void setOutput(String value); +// } +// +// public static void main(String[] args) { +// Options options = PipelineOptionsFactory.fromArgs(args) +// .withValidation() +// .as(Options.class); +// DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); +// +// Pipeline p = Pipeline.create(dataflowOptions); +// +// double samplingThreshold = 0.1; +// +// p.apply(TextIO.Read +// .from(options.getInput()) +// .withCoder(TableRowJsonCoder.of())) +// .apply(new ComputeTopSessions(samplingThreshold)) +// .apply(TextIO.Write.named("Write").withoutSharding().to(options.getOutput())); +// +// p.run(); +// } +//} From c9404ea37283136ae7f87c20e658d22206ee6ad9 Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 16 Oct 2015 15:52:48 +0200 Subject: [PATCH 111/149] move these examples to test and convert them to test cases eventually --- .../dataflow/examples/FlattenizeThis.java | 117 ------------------ .../dataflow/examples/ParDoMultiOutput.java | 89 ------------- .../flink/dataflow/FlattenizeThis.java | 117 ++++++++++++++++++ .../flink/dataflow/ParDoMultiOutput.java | 89 +++++++++++++ 4 files changed, 206 insertions(+), 206 deletions(-) delete mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/FlattenizeThis.java delete mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/ParDoMultiOutput.java create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeThis.java create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutput.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/FlattenizeThis.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/FlattenizeThis.java deleted file mode 100644 index fafff4d4e894..000000000000 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/FlattenizeThis.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 com.dataartisans.flink.dataflow.examples; - -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.google.cloud.dataflow.examples.TfIdf; -import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; -import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.*; -import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.cloud.dataflow.sdk.util.GcsUtil; -import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; -import com.google.cloud.dataflow.sdk.values.KV; - -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -public class FlattenizeThis { - - private static interface Options extends PipelineOptions { - @Description("Path to the directory or GCS prefix containing files to read from") - @Default.String("gs://dataflow-samples/shakespeare/") - String getInput(); - void setInput(String value); - - @Description("Prefix of output URI to write to") - @Validation.Required - String getOutput(); - void setOutput(String value); - } - - public static Set listInputDocuments(Options options) - throws URISyntaxException, IOException { - URI baseUri = new URI(options.getInput()); - - // List all documents in the directory or GCS prefix. - URI absoluteUri; - if (baseUri.getScheme() != null) { - absoluteUri = baseUri; - } else { - absoluteUri = new URI( - "file", - baseUri.getAuthority(), - baseUri.getPath(), - baseUri.getQuery(), - baseUri.getFragment()); - } - - Set uris = new HashSet<>(); - if (absoluteUri.getScheme().equals("file")) { - File directory = new File(absoluteUri); - for (String entry : directory.list()) { - File path = new File(directory, entry); - uris.add(path.toURI()); - } - } else if (absoluteUri.getScheme().equals("gs")) { - GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil(); - URI gcsUriGlob = new URI( - absoluteUri.getScheme(), - absoluteUri.getAuthority(), - absoluteUri.getPath() + "*", - absoluteUri.getQuery(), - absoluteUri.getFragment()); - for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) { - uris.add(entry.toUri()); - } - } - - return uris; - } - - public static void main(String[] args) throws IOException, URISyntaxException { - Options options = PipelineOptionsFactory.create().as(Options.class); - options.setOutput("/tmp/output2.txt"); - options.setInput("/tmp/documents"); - //options.setRunner(DirectPipelineRunner.class); - options.setRunner(FlinkPipelineRunner.class); - - Pipeline p = Pipeline.create(options); - - p.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); - - - p.apply(new TfIdf.ReadDocuments(listInputDocuments(options))) - .apply(ParDo.of(new DoFn, String>() { - @Override - public void processElement(ProcessContext c) throws Exception { - c.output(c.element().toString()); - } - })) - .apply(TextIO.Write.named("WriteCounts") - .to(options.getOutput())); - - p.run(); - } -} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/ParDoMultiOutput.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/ParDoMultiOutput.java deleted file mode 100644 index 5df3d86cff9f..000000000000 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/ParDoMultiOutput.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 com.dataartisans.flink.dataflow.examples; - -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.dataartisans.flink.dataflow.io.ConsoleIO; -import com.google.cloud.dataflow.examples.WordCount.Options; -import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.transforms.Create; -import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.cloud.dataflow.sdk.values.PCollection; -import com.google.cloud.dataflow.sdk.values.PCollectionTuple; -import com.google.cloud.dataflow.sdk.values.TupleTag; -import com.google.cloud.dataflow.sdk.values.TupleTagList; - -public class ParDoMultiOutput { - - public static void main(String[] args) { - - Options options = PipelineOptionsFactory.create().as(Options.class); - options.setOutput("/tmp/output2.txt"); - options.setInput("/tmp/documents/hello_world.txt"); - //options.setRunner(DirectPipelineRunner.class); - options.setRunner(FlinkPipelineRunner.class); - - Pipeline p = Pipeline.create(options); - - PCollection words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO")); - - // Select words whose length is below a cut off, - // plus the lengths of words that are above the cut off. - // Also select words starting with "MARKER". - final int wordLengthCutOff = 3; - // Create tags to use for the main and side outputs. - final TupleTag wordsBelowCutOffTag = new TupleTag(){}; - final TupleTag wordLengthsAboveCutOffTag = new TupleTag(){}; - final TupleTag markedWordsTag = new TupleTag(){}; - - PCollectionTuple results = - words.apply(ParDo - .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag) - .and(markedWordsTag)) - .of(new DoFn() { - final TupleTag specialWordsTag = new TupleTag() { - }; - - public void processElement(ProcessContext c) { - String word = c.element(); - if (word.length() <= wordLengthCutOff) { - c.output(word); - } else { - c.sideOutput(wordLengthsAboveCutOffTag, word.length()); - } - if (word.startsWith("MAA")) { - c.sideOutput(markedWordsTag, word); - } - - if (word.startsWith("SPECIAL")) { - c.sideOutput(specialWordsTag, word); - } - } - })); - - // Extract the PCollection results, by tag. - PCollection wordsBelowCutOff = results.get(wordsBelowCutOffTag); - PCollection wordLengthsAboveCutOff = results.get - (wordLengthsAboveCutOffTag); - PCollection markedWords = results.get(markedWordsTag); - - markedWords.apply(ConsoleIO.Write.create()); - - p.run(); - } -} diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeThis.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeThis.java new file mode 100644 index 000000000000..c9be672ef804 --- /dev/null +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeThis.java @@ -0,0 +1,117 @@ +///* +// * Copyright 2015 Data Artisans GmbH +// * +// * Licensed 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 com.dataartisans.flink.dataflow.examples; +// +//import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +//import com.google.cloud.dataflow.examples.TfIdf; +//import com.google.cloud.dataflow.sdk.Pipeline; +//import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; +//import com.google.cloud.dataflow.sdk.io.TextIO; +//import com.google.cloud.dataflow.sdk.options.*; +//import com.google.cloud.dataflow.sdk.transforms.DoFn; +//import com.google.cloud.dataflow.sdk.transforms.ParDo; +//import com.google.cloud.dataflow.sdk.util.GcsUtil; +//import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; +//import com.google.cloud.dataflow.sdk.values.KV; +// +//import java.io.File; +//import java.io.IOException; +//import java.net.URI; +//import java.net.URISyntaxException; +//import java.util.ArrayList; +//import java.util.HashSet; +//import java.util.List; +//import java.util.Set; +// +//public class FlattenizeThis { +// +// private static interface Options extends PipelineOptions { +// @Description("Path to the directory or GCS prefix containing files to read from") +// @Default.String("gs://dataflow-samples/shakespeare/") +// String getInput(); +// void setInput(String value); +// +// @Description("Prefix of output URI to write to") +// @Validation.Required +// String getOutput(); +// void setOutput(String value); +// } +// +// public static Set listInputDocuments(Options options) +// throws URISyntaxException, IOException { +// URI baseUri = new URI(options.getInput()); +// +// // List all documents in the directory or GCS prefix. +// URI absoluteUri; +// if (baseUri.getScheme() != null) { +// absoluteUri = baseUri; +// } else { +// absoluteUri = new URI( +// "file", +// baseUri.getAuthority(), +// baseUri.getPath(), +// baseUri.getQuery(), +// baseUri.getFragment()); +// } +// +// Set uris = new HashSet<>(); +// if (absoluteUri.getScheme().equals("file")) { +// File directory = new File(absoluteUri); +// for (String entry : directory.list()) { +// File path = new File(directory, entry); +// uris.add(path.toURI()); +// } +// } else if (absoluteUri.getScheme().equals("gs")) { +// GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil(); +// URI gcsUriGlob = new URI( +// absoluteUri.getScheme(), +// absoluteUri.getAuthority(), +// absoluteUri.getPath() + "*", +// absoluteUri.getQuery(), +// absoluteUri.getFragment()); +// for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) { +// uris.add(entry.toUri()); +// } +// } +// +// return uris; +// } +// +// public static void main(String[] args) throws IOException, URISyntaxException { +// Options options = PipelineOptionsFactory.create().as(Options.class); +// options.setOutput("/tmp/output2.txt"); +// options.setInput("/tmp/documents"); +// //options.setRunner(DirectPipelineRunner.class); +// options.setRunner(FlinkPipelineRunner.class); +// +// Pipeline p = Pipeline.create(options); +// +// p.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); +// +// +// p.apply(new Tfidf.ReadDocuments(listInputDocuments(options))) +// .apply(ParDo.of(new DoFn, String>() { +// @Override +// public void processElement(ProcessContext c) throws Exception { +// c.output(c.element().toString()); +// } +// })) +// .apply(TextIO.Write.named("WriteCounts") +// .to(options.getOutput())); +// +// p.run(); +// } +//} diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutput.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutput.java new file mode 100644 index 000000000000..7365777b7609 --- /dev/null +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutput.java @@ -0,0 +1,89 @@ +///* +// * Copyright 2015 Data Artisans GmbH +// * +// * Licensed 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 com.dataartisans.flink.dataflow.examples; +// +//import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +//import com.dataartisans.flink.dataflow.io.ConsoleIO; +//import com.google.cloud.dataflow.examples.WordCount.Options; +//import com.google.cloud.dataflow.sdk.Pipeline; +//import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +//import com.google.cloud.dataflow.sdk.transforms.Create; +//import com.google.cloud.dataflow.sdk.transforms.DoFn; +//import com.google.cloud.dataflow.sdk.transforms.ParDo; +//import com.google.cloud.dataflow.sdk.values.PCollection; +//import com.google.cloud.dataflow.sdk.values.PCollectionTuple; +//import com.google.cloud.dataflow.sdk.values.TupleTag; +//import com.google.cloud.dataflow.sdk.values.TupleTagList; +// +//public class ParDoMultiOutput { +// +// public static void main(String[] args) { +// +// Options options = PipelineOptionsFactory.create().as(Options.class); +// options.setOutput("/tmp/output2.txt"); +// options.setInput("/tmp/documents/hello_world.txt"); +// //options.setRunner(DirectPipelineRunner.class); +// options.setRunner(FlinkPipelineRunner.class); +// +// Pipeline p = Pipeline.create(options); +// +// PCollection words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO")); +// +// // Select words whose length is below a cut off, +// // plus the lengths of words that are above the cut off. +// // Also select words starting with "MARKER". +// final int wordLengthCutOff = 3; +// // Create tags to use for the main and side outputs. +// final TupleTag wordsBelowCutOffTag = new TupleTag(){}; +// final TupleTag wordLengthsAboveCutOffTag = new TupleTag(){}; +// final TupleTag markedWordsTag = new TupleTag(){}; +// +// PCollectionTuple results = +// words.apply(ParDo +// .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag) +// .and(markedWordsTag)) +// .of(new DoFn() { +// final TupleTag specialWordsTag = new TupleTag() { +// }; +// +// public void processElement(ProcessContext c) { +// String word = c.element(); +// if (word.length() <= wordLengthCutOff) { +// c.output(word); +// } else { +// c.sideOutput(wordLengthsAboveCutOffTag, word.length()); +// } +// if (word.startsWith("MAA")) { +// c.sideOutput(markedWordsTag, word); +// } +// +// if (word.startsWith("SPECIAL")) { +// c.sideOutput(specialWordsTag, word); +// } +// } +// })); +// +// // Extract the PCollection results, by tag. +// PCollection wordsBelowCutOff = results.get(wordsBelowCutOffTag); +// PCollection wordLengthsAboveCutOff = results.get +// (wordLengthsAboveCutOffTag); +// PCollection markedWords = results.get(markedWordsTag); +// +// markedWords.apply(ConsoleIO.Write.create()); +// +// p.run(); +// } +//} From aa1a933e4c37aa6a515b41cff6c9ff815f7ed420 Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 4 Nov 2015 17:14:15 +0100 Subject: [PATCH 112/149] move JoinExamples to util --- .../com/dataartisans/flink/dataflow/JoinExamplesITCase.java | 2 ++ .../dataartisans/flink/dataflow/{ => util}/JoinExamples.java | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) rename runners/flink/src/test/java/com/dataartisans/flink/dataflow/{ => util}/JoinExamples.java (98%) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java index 3e8109423312..dfcadc19a861 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java @@ -15,6 +15,7 @@ */ package com.dataartisans.flink.dataflow; +import com.dataartisans.flink.dataflow.util.JoinExamples; import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; @@ -76,6 +77,7 @@ public JoinExamplesITCase(){ "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, " + "url: http://cnn.com" }; + @Override protected void preSubmit() throws Exception { resultPath = getTempDirPath("result"); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java similarity index 98% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamples.java rename to runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java index 648e54804648..39e4a86cb7ee 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamples.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package com.dataartisans.flink.dataflow.util; import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.dataflow.sdk.Pipeline; @@ -48,7 +48,7 @@ public class JoinExamples { /** * Join two collections, using country code as the key. */ - static PCollection joinEvents(PCollection eventsTable, + public static PCollection joinEvents(PCollection eventsTable, PCollection countryCodes) throws Exception { final TupleTag eventInfoTag = new TupleTag(); From d5881a58761ab31bccee149fa999df56fbf1b1bd Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 4 Nov 2015 17:14:59 +0100 Subject: [PATCH 113/149] created it case Flattenize --- .../flink/dataflow/FlattenizeITCase.java | 72 +++++++++++ .../flink/dataflow/FlattenizeThis.java | 117 ------------------ 2 files changed, 72 insertions(+), 117 deletions(-) create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java delete mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeThis.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java new file mode 100644 index 000000000000..bc2451498a42 --- /dev/null +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java @@ -0,0 +1,72 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.Flatten; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; +import com.google.common.base.Joiner; +import org.apache.flink.test.util.JavaProgramTestBase; + +public class FlattenizeITCase extends JavaProgramTestBase { + + private String resultPath; + private String resultPath2; + + private static final String[] words = {"hello", "this", "is", "a", "DataSet!"}; + private static final String[] words2 = {"hello", "this", "is", "another", "DataSet!"}; + private static final String[] words3 = {"hello", "this", "is", "yet", "another", "DataSet!"}; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + resultPath2 = getTempDirPath("result2"); + } + + @Override + protected void postSubmit() throws Exception { + String join = Joiner.on('\n').join(words); + String join2 = Joiner.on('\n').join(words2); + String join3 = Joiner.on('\n').join(words3); + compareResultsByLinesInMemory(join + "\n" + join2, resultPath); + compareResultsByLinesInMemory(join + "\n" + join2 + "\n" + join3, resultPath2); + } + + + @Override + protected void testProgram() throws Exception { + Pipeline p = FlinkTestPipeline.create(); + + PCollection p1 = p.apply(Create.of(words)); + PCollection p2 = p.apply(Create.of(words2)); + + PCollectionList list = PCollectionList.of(p1).and(p2); + + list.apply(Flatten.pCollections()).apply(TextIO.Write.to(resultPath)); + + PCollection p3 = p.apply(Create.of(words3)); + + PCollectionList list2 = list.and(p3); + + list2.apply(Flatten.pCollections()).apply(TextIO.Write.to(resultPath2)); + + p.run(); + } + +} diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeThis.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeThis.java deleted file mode 100644 index c9be672ef804..000000000000 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeThis.java +++ /dev/null @@ -1,117 +0,0 @@ -///* -// * Copyright 2015 Data Artisans GmbH -// * -// * Licensed 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 com.dataartisans.flink.dataflow.examples; -// -//import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -//import com.google.cloud.dataflow.examples.TfIdf; -//import com.google.cloud.dataflow.sdk.Pipeline; -//import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; -//import com.google.cloud.dataflow.sdk.io.TextIO; -//import com.google.cloud.dataflow.sdk.options.*; -//import com.google.cloud.dataflow.sdk.transforms.DoFn; -//import com.google.cloud.dataflow.sdk.transforms.ParDo; -//import com.google.cloud.dataflow.sdk.util.GcsUtil; -//import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath; -//import com.google.cloud.dataflow.sdk.values.KV; -// -//import java.io.File; -//import java.io.IOException; -//import java.net.URI; -//import java.net.URISyntaxException; -//import java.util.ArrayList; -//import java.util.HashSet; -//import java.util.List; -//import java.util.Set; -// -//public class FlattenizeThis { -// -// private static interface Options extends PipelineOptions { -// @Description("Path to the directory or GCS prefix containing files to read from") -// @Default.String("gs://dataflow-samples/shakespeare/") -// String getInput(); -// void setInput(String value); -// -// @Description("Prefix of output URI to write to") -// @Validation.Required -// String getOutput(); -// void setOutput(String value); -// } -// -// public static Set listInputDocuments(Options options) -// throws URISyntaxException, IOException { -// URI baseUri = new URI(options.getInput()); -// -// // List all documents in the directory or GCS prefix. -// URI absoluteUri; -// if (baseUri.getScheme() != null) { -// absoluteUri = baseUri; -// } else { -// absoluteUri = new URI( -// "file", -// baseUri.getAuthority(), -// baseUri.getPath(), -// baseUri.getQuery(), -// baseUri.getFragment()); -// } -// -// Set uris = new HashSet<>(); -// if (absoluteUri.getScheme().equals("file")) { -// File directory = new File(absoluteUri); -// for (String entry : directory.list()) { -// File path = new File(directory, entry); -// uris.add(path.toURI()); -// } -// } else if (absoluteUri.getScheme().equals("gs")) { -// GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil(); -// URI gcsUriGlob = new URI( -// absoluteUri.getScheme(), -// absoluteUri.getAuthority(), -// absoluteUri.getPath() + "*", -// absoluteUri.getQuery(), -// absoluteUri.getFragment()); -// for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) { -// uris.add(entry.toUri()); -// } -// } -// -// return uris; -// } -// -// public static void main(String[] args) throws IOException, URISyntaxException { -// Options options = PipelineOptionsFactory.create().as(Options.class); -// options.setOutput("/tmp/output2.txt"); -// options.setInput("/tmp/documents"); -// //options.setRunner(DirectPipelineRunner.class); -// options.setRunner(FlinkPipelineRunner.class); -// -// Pipeline p = Pipeline.create(options); -// -// p.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); -// -// -// p.apply(new Tfidf.ReadDocuments(listInputDocuments(options))) -// .apply(ParDo.of(new DoFn, String>() { -// @Override -// public void processElement(ProcessContext c) throws Exception { -// c.output(c.element().toString()); -// } -// })) -// .apply(TextIO.Write.named("WriteCounts") -// .to(options.getOutput())); -// -// p.run(); -// } -//} From 189ae44aa40382c17066bfe55567446858113f0e Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 4 Nov 2015 17:16:18 +0100 Subject: [PATCH 114/149] optimize imports --- .../flink/dataflow/FlinkRunnerResult.java | 1 - .../flink/dataflow/examples/TFIDF.java | 1 - .../translation/FlinkPipelineTranslator.java | 1 - .../FlinkTransformTranslators.java | 21 +++++++++++++++++-- .../translation/TranslationContext.java | 2 -- .../functions/FlinkDoFnFunction.java | 7 ------- .../FlinkMultiOutputDoFnFunction.java | 3 --- .../functions/FlinkPartialReduceFunction.java | 1 - .../translation/functions/UnionCoder.java | 8 +++---- .../types/CoderTypeInformation.java | 4 ---- .../translation/types/KvCoderComperator.java | 1 - .../types/KvCoderTypeInformation.java | 2 -- .../wrappers/CombineFnAggregatorWrapper.java | 6 ------ .../wrappers/SourceInputFormat.java | 3 --- .../wrappers/SourceInputSplit.java | 5 ----- .../flink/dataflow/MaybeEmptyTestITCase.java | 3 --- .../flink/dataflow/ReadSourceITCase.java | 1 - .../flink/dataflow/WordCountJoin2ITCase.java | 5 ++++- .../flink/dataflow/WordCountJoin3ITCase.java | 5 ++++- 19 files changed, 30 insertions(+), 50 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java index 92185e234e21..59b8b63709ed 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java @@ -19,7 +19,6 @@ import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException; import com.google.cloud.dataflow.sdk.runners.AggregatorValues; import com.google.cloud.dataflow.sdk.transforms.Aggregator; -import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsNotFound; import java.util.Collections; import java.util.Map; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java index 60146c9ef357..88422ee22dc9 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java @@ -52,7 +52,6 @@ import com.google.cloud.dataflow.sdk.values.PDone; import com.google.cloud.dataflow.sdk.values.PInput; import com.google.cloud.dataflow.sdk.values.TupleTag; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java index 885a7d0f914b..92b9135cccea 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java @@ -22,7 +22,6 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; -import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; import com.google.cloud.dataflow.sdk.values.PValue; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 43f32fd90013..874412222c69 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -18,7 +18,15 @@ package com.dataartisans.flink.dataflow.translation; import com.dataartisans.flink.dataflow.io.ConsoleIO; -import com.dataartisans.flink.dataflow.translation.functions.*; +import com.dataartisans.flink.dataflow.translation.functions.FlinkCoGroupKeyedListAggregator; +import com.dataartisans.flink.dataflow.translation.functions.FlinkCreateFunction; +import com.dataartisans.flink.dataflow.translation.functions.FlinkDoFnFunction; +import com.dataartisans.flink.dataflow.translation.functions.FlinkKeyedListAggregationFunction; +import com.dataartisans.flink.dataflow.translation.functions.FlinkMultiOutputDoFnFunction; +import com.dataartisans.flink.dataflow.translation.functions.FlinkMultiOutputPruningFunction; +import com.dataartisans.flink.dataflow.translation.functions.FlinkPartialReduceFunction; +import com.dataartisans.flink.dataflow.translation.functions.FlinkReduceFunction; +import com.dataartisans.flink.dataflow.translation.functions.UnionCoder; import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation; import com.dataartisans.flink.dataflow.translation.wrappers.SourceInputFormat; @@ -55,7 +63,16 @@ import org.apache.flink.api.java.io.AvroInputFormat; import org.apache.flink.api.java.io.AvroOutputFormat; import org.apache.flink.api.java.io.TextInputFormat; -import org.apache.flink.api.java.operators.*; +import org.apache.flink.api.java.operators.CoGroupOperator; +import org.apache.flink.api.java.operators.DataSink; +import org.apache.flink.api.java.operators.DataSource; +import org.apache.flink.api.java.operators.FlatMapOperator; +import org.apache.flink.api.java.operators.GroupCombineOperator; +import org.apache.flink.api.java.operators.GroupReduceOperator; +import org.apache.flink.api.java.operators.Grouping; +import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.java.operators.MapPartitionOperator; +import org.apache.flink.api.java.operators.UnsortedGrouping; import org.apache.flink.core.fs.Path; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index 7f53e2c41535..de1a19cf02e2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -37,8 +37,6 @@ import java.util.HashMap; import java.util.Map; -import static com.google.common.base.Preconditions.checkArgument; - public class TranslationContext { private final Map> dataSets; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java index 02af91d62cdc..7eb8cdfabed4 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java @@ -15,17 +15,13 @@ */ package com.dataartisans.flink.dataflow.translation.functions; -import com.dataartisans.flink.dataflow.translation.wrappers.CombineFnAggregatorWrapper; import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.BooleanNode; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.repackaged.com.google.common.collect.Lists; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; @@ -36,7 +32,6 @@ import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import com.google.common.collect.ImmutableList; -import javafx.scene.layout.Pane; import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.util.Collector; import org.joda.time.Instant; @@ -46,8 +41,6 @@ import java.io.ObjectOutputStream; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; -import java.util.LinkedList; import java.util.List; /** diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java index 84f3e27975ec..c111a8c0e1b7 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -15,14 +15,12 @@ */ package com.dataartisans.flink.dataflow.translation.functions; -import com.dataartisans.flink.dataflow.translation.wrappers.CombineFnAggregatorWrapper; import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; @@ -40,7 +38,6 @@ import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.util.ArrayList; -import java.util.Collection; import java.util.List; import java.util.Map; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java index 053808d6fb99..ef47b7257a80 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java @@ -20,7 +20,6 @@ import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.values.KV; import org.apache.flink.api.common.functions.GroupCombineFunction; -import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.util.Collector; import java.util.Iterator; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java index cd7e04dcb8ba..c0a72c5a53f3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java @@ -17,18 +17,16 @@ package com.dataartisans.flink.dataflow.translation.functions; -import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; - +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.coders.StandardCoder; +import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.util.VarInt; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java index e249a6cb73b2..2d487b483c41 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java @@ -18,18 +18,14 @@ package com.dataartisans.flink.dataflow.translation.types; import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.shaded.com.google.common.base.Preconditions; -import java.util.List; - /** * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s. diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index 887452ed4345..a349953ffbfc 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -26,7 +26,6 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.MemorySegment; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.ObjectInputStream; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java index 5935bf9c481b..fc1e76c87946 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java @@ -17,9 +17,7 @@ */ package com.dataartisans.flink.dataflow.translation.types; -import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; -import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.values.KV; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java index 651cfe629565..924b29769f0f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java @@ -19,15 +19,9 @@ import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; -import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.common.collect.Lists; import org.apache.flink.api.common.accumulators.Accumulator; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.io.Serializable; /** diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java index eb1ca4b19f7e..8c9c59ca54b1 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java @@ -17,15 +17,12 @@ */ package com.dataartisans.flink.dataflow.translation.wrappers; -import com.dataartisans.flink.dataflow.io.ConsoleIO; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.util.Lists; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.BoundedSource; -import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.Source; import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.util.WindowedValue; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java index 8dc75735be61..2b93ab78e65c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java @@ -18,12 +18,7 @@ package com.dataartisans.flink.dataflow.translation.wrappers; import com.google.cloud.dataflow.sdk.io.Source; -import com.google.cloud.dataflow.sdk.util.SerializableUtils; import org.apache.flink.core.io.InputSplit; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - -import java.io.IOException; /** * {@link org.apache.flink.core.io.InputSplit} for diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java index d8b392922eaa..29c34d4b3365 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java @@ -15,15 +15,12 @@ */ package com.dataartisans.flink.dataflow; -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.*; import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.common.base.Joiner; import org.apache.flink.test.util.JavaProgramTestBase; import java.io.Serializable; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java index 37d9221cca65..c4dafaa6731f 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java @@ -21,7 +21,6 @@ import com.google.cloud.dataflow.sdk.io.BoundedSource; import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java index e1019c53192a..c3eed61d774b 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java @@ -17,7 +17,10 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.transforms.*; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java index fcdf107c1389..33e67cc234e8 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java @@ -17,7 +17,10 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.transforms.*; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; From c36ca1410de2f23a58b1224363bd19f7577b5914 Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 4 Nov 2015 17:24:54 +0100 Subject: [PATCH 115/149] create ParDoMultiOutput IT case --- .../flink/dataflow/ParDoMultiOutput.java | 89 ----------------- .../dataflow/ParDoMultiOutputITCase.java | 98 +++++++++++++++++++ 2 files changed, 98 insertions(+), 89 deletions(-) delete mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutput.java create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutput.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutput.java deleted file mode 100644 index 7365777b7609..000000000000 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutput.java +++ /dev/null @@ -1,89 +0,0 @@ -///* -// * Copyright 2015 Data Artisans GmbH -// * -// * Licensed 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 com.dataartisans.flink.dataflow.examples; -// -//import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -//import com.dataartisans.flink.dataflow.io.ConsoleIO; -//import com.google.cloud.dataflow.examples.WordCount.Options; -//import com.google.cloud.dataflow.sdk.Pipeline; -//import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -//import com.google.cloud.dataflow.sdk.transforms.Create; -//import com.google.cloud.dataflow.sdk.transforms.DoFn; -//import com.google.cloud.dataflow.sdk.transforms.ParDo; -//import com.google.cloud.dataflow.sdk.values.PCollection; -//import com.google.cloud.dataflow.sdk.values.PCollectionTuple; -//import com.google.cloud.dataflow.sdk.values.TupleTag; -//import com.google.cloud.dataflow.sdk.values.TupleTagList; -// -//public class ParDoMultiOutput { -// -// public static void main(String[] args) { -// -// Options options = PipelineOptionsFactory.create().as(Options.class); -// options.setOutput("/tmp/output2.txt"); -// options.setInput("/tmp/documents/hello_world.txt"); -// //options.setRunner(DirectPipelineRunner.class); -// options.setRunner(FlinkPipelineRunner.class); -// -// Pipeline p = Pipeline.create(options); -// -// PCollection words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO")); -// -// // Select words whose length is below a cut off, -// // plus the lengths of words that are above the cut off. -// // Also select words starting with "MARKER". -// final int wordLengthCutOff = 3; -// // Create tags to use for the main and side outputs. -// final TupleTag wordsBelowCutOffTag = new TupleTag(){}; -// final TupleTag wordLengthsAboveCutOffTag = new TupleTag(){}; -// final TupleTag markedWordsTag = new TupleTag(){}; -// -// PCollectionTuple results = -// words.apply(ParDo -// .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag) -// .and(markedWordsTag)) -// .of(new DoFn() { -// final TupleTag specialWordsTag = new TupleTag() { -// }; -// -// public void processElement(ProcessContext c) { -// String word = c.element(); -// if (word.length() <= wordLengthCutOff) { -// c.output(word); -// } else { -// c.sideOutput(wordLengthsAboveCutOffTag, word.length()); -// } -// if (word.startsWith("MAA")) { -// c.sideOutput(markedWordsTag, word); -// } -// -// if (word.startsWith("SPECIAL")) { -// c.sideOutput(specialWordsTag, word); -// } -// } -// })); -// -// // Extract the PCollection results, by tag. -// PCollection wordsBelowCutOff = results.get(wordsBelowCutOffTag); -// PCollection wordLengthsAboveCutOff = results.get -// (wordLengthsAboveCutOffTag); -// PCollection markedWords = results.get(markedWordsTag); -// -// markedWords.apply(ConsoleIO.Write.create()); -// -// p.run(); -// } -//} diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java new file mode 100644 index 000000000000..dbe88d2cfbcf --- /dev/null +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java @@ -0,0 +1,98 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionTuple; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.cloud.dataflow.sdk.values.TupleTagList; +import com.google.common.base.Joiner; +import org.apache.flink.test.util.JavaProgramTestBase; + +import java.io.Serializable; + +public class ParDoMultiOutputITCase extends JavaProgramTestBase implements Serializable { + + private String resultPath; + + private static String[] expectedWords = {"MAAA", "MAAFOOO"}; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on("\n").join(expectedWords), resultPath); + } + + @Override + protected void testProgram() throws Exception { + Pipeline p = FlinkTestPipeline.create(); + + PCollection words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO")); + + // Select words whose length is below a cut off, + // plus the lengths of words that are above the cut off. + // Also select words starting with "MARKER". + final int wordLengthCutOff = 3; + // Create tags to use for the main and side outputs. + final TupleTag wordsBelowCutOffTag = new TupleTag(){}; + final TupleTag wordLengthsAboveCutOffTag = new TupleTag(){}; + final TupleTag markedWordsTag = new TupleTag(){}; + + PCollectionTuple results = + words.apply(ParDo + .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag) + .and(markedWordsTag)) + .of(new DoFn() { + final TupleTag specialWordsTag = new TupleTag() { + }; + + public void processElement(ProcessContext c) { + String word = c.element(); + if (word.length() <= wordLengthCutOff) { + c.output(word); + } else { + c.sideOutput(wordLengthsAboveCutOffTag, word.length()); + } + if (word.startsWith("MAA")) { + c.sideOutput(markedWordsTag, word); + } + + if (word.startsWith("SPECIAL")) { + c.sideOutput(specialWordsTag, word); + } + } + })); + + // Extract the PCollection results, by tag. + PCollection wordsBelowCutOff = results.get(wordsBelowCutOffTag); + PCollection wordLengthsAboveCutOff = results.get + (wordLengthsAboveCutOffTag); + PCollection markedWords = results.get(markedWordsTag); + + markedWords.apply(TextIO.Write.to(resultPath)); + + p.run(); + } +} From 2e563f08309f8b1c819abdee76782584d8453aac Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 16 Nov 2015 12:25:57 +0100 Subject: [PATCH 116/149] port to Flink 0.10.0 --- runners/flink/pom.xml | 4 +- .../FlinkTransformTranslators.java | 13 ++----- .../types/CoderTypeInformation.java | 5 +++ .../types/CoderTypeSerializer.java | 5 +++ .../translation/types/KvCoderComperator.java | 1 + .../types/KvCoderTypeInformation.java | 39 +++++++++++++------ .../types/VoidCoderTypeSerializer.java | 22 +++++++++++ 7 files changed, 66 insertions(+), 23 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index abccc77239e8..5704f08066b9 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -21,7 +21,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans com.dataartisans flink-dataflow - 0.1 + 0.2 Flink Dataflow Runner jar @@ -39,7 +39,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans UTF-8 UTF-8 - 0.9.1 + 0.10.0 com.dataartisans.flink.dataflow.examples.WordCount diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 874412222c69..26a580fed4aa 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -117,17 +117,11 @@ public class FlinkTransformTranslators { TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator()); TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslator()); + TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslator()); + TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslator()); TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslator()); - //TRANSLATORS.put(BigQueryIO.Read.Bound.class, null); - //TRANSLATORS.put(BigQueryIO.Write.Bound.class, null); - - //TRANSLATORS.put(DatastoreIO.Sink.class, null); - - //TRANSLATORS.put(PubsubIO.Read.Bound.class, null); - //TRANSLATORS.put(PubsubIO.Write.Bound.class, null); - TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslator()); // TRANSLATORS.put(Write.Bound.class, new ReadSourceTranslator()); @@ -136,8 +130,7 @@ public class FlinkTransformTranslators { // Flink-specific TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslator()); - - TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslator()); + } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java index 2d487b483c41..56192cd8e2aa 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java @@ -97,6 +97,11 @@ public int hashCode() { return coder.hashCode(); } + @Override + public boolean canEqual(Object obj) { + return obj instanceof CoderTypeInformation; + } + @Override public String toString() { return "CoderTypeInformation{" + diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java index 58357cd9d0c6..cb8f6d304fbb 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java @@ -143,6 +143,11 @@ public boolean equals(Object o) { return true; } + @Override + public boolean canEqual(Object obj) { + return obj instanceof CoderTypeSerializer; + } + @Override public int hashCode() { return coder.hashCode(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index a349953ffbfc..940dba6b85f4 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -159,6 +159,7 @@ public int compare(KV first, KV second) { @Override public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { + inputWrapper.setInputView(firstSource); K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); inputWrapper.setInputView(secondSource); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java index fc1e76c87946..3e679a436dbb 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java @@ -18,14 +18,19 @@ package com.dataartisans.flink.dataflow.translation.types; import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.options.Default; import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.primitives.Ints; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.runtime.TupleComparator; import org.apache.flink.shaded.com.google.common.base.Preconditions; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; /** @@ -36,10 +41,12 @@ public class KvCoderTypeInformation extends CompositeType> { private KvCoder coder; + // We don't have the Class, so we have to pass null here. What a shame... + private static Object DUMMY = new Object(); + @SuppressWarnings("unchecked") public KvCoderTypeInformation(KvCoder coder) { - // We don't have the Class, so we have to pass null here. What a shame... - super(null); + super(((Class>) DUMMY.getClass())); this.coder = coder; Preconditions.checkNotNull(coder); } @@ -153,18 +160,28 @@ public int getFieldIndex(String fieldName) { } } - // These three we only have because we support CompositeType, we create our own comparator - // in createComparator. - @Override - protected void initializeNewComparator(int localKeyCount) {} - @Override - protected void addCompareField(int fieldId, TypeComparator comparator) {} - @Override - protected TypeComparator> getNewComparator(ExecutionConfig config) { return null; } - @Override public void getFlatFields(String fieldExpression, int offset, List result) { CoderTypeInformation keyTypeInfo = new CoderTypeInformation<>(coder.getKeyCoder()); result.add(new FlatFieldDescriptor(0, keyTypeInfo)); } + + @Override + protected TypeComparatorBuilder> createTypeComparatorBuilder() { + return new KvCoderTypeComparatorBuilder(); + } + + private class KvCoderTypeComparatorBuilder implements TypeComparatorBuilder> { + + @Override + public void initializeTypeComparatorBuilder(int size) {} + + @Override + public void addComparatorField(int fieldId, TypeComparator comparator) {} + + @Override + public TypeComparator> createTypeComparator(ExecutionConfig config) { + return new KvCoderComperator<>(coder); + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java index abad14b1721e..f4b6275963b0 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java @@ -17,7 +17,9 @@ */ package com.dataartisans.flink.dataflow.translation.types; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.runtime.ValueSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -83,6 +85,26 @@ public void copy(DataInputView source, DataOutputView target) throws IOException target.writeByte(1); } + @Override + public boolean equals(Object obj) { + if (obj instanceof VoidCoderTypeSerializer) { + VoidCoderTypeSerializer other = (VoidCoderTypeSerializer) obj; + return other.canEqual(this); + } else { + return false; + } + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof VoidCoderTypeSerializer; + } + + @Override + public int hashCode() { + return 0; + } + public static class VoidValue { private VoidValue() {} From 35f104fb9888088b14842693f58da10d894c0113 Mon Sep 17 00:00:00 2001 From: smarthi Date: Tue, 22 Dec 2015 20:48:32 -0500 Subject: [PATCH 117/149] Enforce Java >= 1.7, Optimize Imports, Java 7 code compatibility --- runners/flink/pom.xml | 5 ++- .../flink/dataflow/FlinkPipelineRunner.java | 2 +- .../flink/dataflow/examples/TFIDF.java | 17 +++++----- .../flink/dataflow/examples/WordCount.java | 2 +- .../FlinkTransformTranslators.java | 8 +++-- .../translation/TranslationContext.java | 8 ++--- .../functions/FlinkDoFnFunction.java | 2 +- .../FlinkMultiOutputDoFnFunction.java | 2 +- .../translation/functions/UnionCoder.java | 5 ++- .../translation/types/CoderComperator.java | 2 +- .../types/CoderTypeSerializer.java | 3 +- .../types/KvCoderTypeInformation.java | 33 +++++++++---------- .../types/VoidCoderTypeSerializer.java | 2 -- .../dataflow/RemoveDuplicatesEmptyITCase.java | 4 +-- .../flink/dataflow/util/JoinExamples.java | 11 +++---- 15 files changed, 51 insertions(+), 55 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 5704f08066b9..288de1be3ada 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -39,7 +39,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans UTF-8 UTF-8 - 0.10.0 + 0.10.1 com.dataartisans.flink.dataflow.examples.WordCount @@ -192,6 +192,9 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans + + [1.7,) + [3.0.3,) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java index 6208e728fc5e..ae31f48fc3ac 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java @@ -178,7 +178,7 @@ public FlinkRunnerResult run(Pipeline pipeline) { LOG.info("Starting execution of Flink program."); - JobExecutionResult result = null; + JobExecutionResult result; try { result = flinkEnv.execute(); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java index 88422ee22dc9..86a269583f2f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java @@ -94,7 +94,7 @@ public class TFIDF { *

* Inherits standard configuration options. */ - private static interface Options extends PipelineOptions, FlinkPipelineOptions { + private interface Options extends PipelineOptions, FlinkPipelineOptions { @Description("Path to the directory or GCS prefix containing files to read from") @Default.String("gs://dataflow-samples/shakespeare/") String getInput(); @@ -293,8 +293,8 @@ public void processElement(ProcessContext c) { // a tuple tag. Each input must have the same key type, URI // in this case. The type parameter of the tuple tag matches // the types of the values for each collection. - final TupleTag wordTotalsTag = new TupleTag(); - final TupleTag> wordCountsTag = new TupleTag>(); + final TupleTag wordTotalsTag = new TupleTag<>(); + final TupleTag> wordCountsTag = new TupleTag<>(); KeyedPCollectionTuple coGbkInput = KeyedPCollectionTuple .of(wordTotalsTag, uriToWordTotal) .and(wordCountsTag, uriToWordAndCount); @@ -361,8 +361,8 @@ public void processElement(ProcessContext c) { // Join the term frequency and document frequency // collections, each keyed on the word. - final TupleTag> tfTag = new TupleTag>(); - final TupleTag dfTag = new TupleTag(); + final TupleTag> tfTag = new TupleTag<>(); + final TupleTag dfTag = new TupleTag<>(); PCollection> wordToUriAndTfAndDf = KeyedPCollectionTuple .of(tfTag, wordToUriAndTf) .and(dfTag, wordToDf) @@ -373,10 +373,11 @@ public void processElement(ProcessContext c) { // ("term frequency - inverse document frequency") score; // here we use a basic version that is the term frequency // divided by the log of the document frequency. - PCollection>> wordToUriAndTfIdf = wordToUriAndTfAndDf + + return wordToUriAndTfAndDf .apply(ParDo.named("ComputeTfIdf").of( new DoFn, KV>>() { - private static final long serialVersionUID = 0; + private static final long serialVersionUID1 = 0; @Override public void processElement(ProcessContext c) { @@ -391,8 +392,6 @@ public void processElement(ProcessContext c) { } } })); - - return wordToUriAndTfIdf; } // Instantiate Logger. diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java index 34ad5c400d40..82f1e46302c2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java @@ -33,7 +33,7 @@ public class WordCount { *

* Inherits standard configuration options. */ - public static interface Options extends PipelineOptions, FlinkPipelineOptions { + public interface Options extends PipelineOptions, FlinkPipelineOptions { @Description("Path of the file to read from") @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") String getInput(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 26a580fed4aa..10a77ec045e9 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -159,6 +159,7 @@ private static class AvroIOReadTranslator implements FlinkPipelineTranslator. private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslator.class); @Override + @SuppressWarnings("unchecked") public void translateNode(AvroIO.Read.Bound transform, TranslationContext context) { String path = transform.getFilepattern(); String name = transform.getName(); @@ -188,6 +189,7 @@ private static class AvroIOWriteTranslator implements FlinkPipelineTranslator private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslator.class); @Override + @SuppressWarnings("unchecked") public void translateNode(AvroIO.Write.Bound transform, TranslationContext context) { DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); String filenamePrefix = transform.getFilenamePrefix(); @@ -211,7 +213,7 @@ public void translateNode(AvroIO.Write.Bound transform, TranslationContext co System.out.println("Could not access type from AvroIO.Bound: " + e); } - DataSink dataSink = inputDataSet.output(new AvroOutputFormat(new Path + DataSink dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path (filenamePrefix), avroType)); if (numShards > 0) { @@ -235,7 +237,7 @@ public void translateNode(TextIO.Read.Bound transform, TranslationContex LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType); LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation); - PValue output = (PValue) context.getOutput(transform); + PValue output = context.getOutput(transform); TypeInformation typeInformation = context.getTypeInfo(output); @@ -276,7 +278,7 @@ public void translateNode(TextIO.Write.Bound transform, TranslationContext co private static class ConsoleIOWriteTranslator implements FlinkPipelineTranslator.TransformTranslator { @Override public void translateNode(ConsoleIO.Write.Bound transform, TranslationContext context) { - PValue input = (PValue) context.getInput(transform); + PValue input = context.getInput(transform); DataSet inputDataSet = context.getInputDataSet(input); inputDataSet.printOnTaskManager(transform.getName()); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java index de1a19cf02e2..af461094fa3f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java @@ -106,7 +106,7 @@ public TypeInformation getTypeInfo(PInput output) { return new CoderTypeInformation(outputCoder); } } - return new GenericTypeInfo((Class)Object.class); + return new GenericTypeInfo<>((Class)Object.class); } public TypeInformation getInputTypeInfo() { @@ -119,13 +119,11 @@ public TypeInformation getOutputTypeInfo() { @SuppressWarnings("unchecked") I getInput(PTransform transform) { - I input = (I) currentTransform.getInput(); - return input; + return (I) currentTransform.getInput(); } @SuppressWarnings("unchecked") O getOutput(PTransform transform) { - O output = (O) currentTransform.getOutput(); - return output; + return (O) currentTransform.getOutput(); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java index 7eb8cdfabed4..1ce0b426c820 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java @@ -188,7 +188,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times @Override protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { - SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper(combiner); + SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper<>(combiner); getRuntimeContext().addAccumulator(name, wrapper); return wrapper; } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java index c111a8c0e1b7..6187182c34cc 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -166,7 +166,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times @Override protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { - SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper(combiner); + SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper<>(combiner); getRuntimeContext().addAccumulator(name, wrapper); return null; } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java index c0a72c5a53f3..0befa8841ff6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java @@ -20,7 +20,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.coders.StandardCoder; import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import com.google.cloud.dataflow.sdk.util.PropertyNames; @@ -75,7 +74,7 @@ public void encode( RawUnionValue union, OutputStream outStream, Context context) - throws IOException, CoderException { + throws IOException { int index = getIndexForEncoding(union); // Write out the union tag. VarInt.encode(index, outStream); @@ -90,7 +89,7 @@ public void encode( @Override public RawUnionValue decode(InputStream inStream, Context context) - throws IOException, CoderException { + throws IOException { int index = VarInt.decodeInt(inStream); Object value = elementCoders.get(index).decode(inStream, context); return new RawUnionValue(index, value); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java index 38569824dbfb..ade826d82c83 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java @@ -139,7 +139,7 @@ public int compare(T first, T second) { @Override public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { - CoderTypeSerializer serializer = new CoderTypeSerializer(coder); + CoderTypeSerializer serializer = new CoderTypeSerializer<>(coder); T first = serializer.deserialize(firstSource); T second = serializer.deserialize(secondSource); return compare(first, second); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java index cb8f6d304fbb..971547764ecf 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java @@ -138,9 +138,8 @@ public boolean equals(Object o) { CoderTypeSerializer that = (CoderTypeSerializer) o; - if (!coder.equals(that.coder)) return false; + return coder.equals(that.coder); - return true; } @Override diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java index 3e679a436dbb..4c72237aeb75 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java @@ -18,19 +18,14 @@ package com.dataartisans.flink.dataflow.translation.types; import com.google.cloud.dataflow.sdk.coders.KvCoder; -import com.google.cloud.dataflow.sdk.options.Default; import com.google.cloud.dataflow.sdk.values.KV; -import com.google.common.primitives.Ints; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.runtime.TupleComparator; import org.apache.flink.shaded.com.google.common.base.Preconditions; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; /** @@ -123,6 +118,7 @@ public String toString() { } @Override + @SuppressWarnings("unchecked") public TypeInformation getTypeAt(int pos) { if (pos == 0) { return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); @@ -134,13 +130,15 @@ public TypeInformation getTypeAt(int pos) { } @Override + @SuppressWarnings("unchecked") public TypeInformation getTypeAt(String fieldExpression) { - if (fieldExpression.equals("key")) { - return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); - } else if (fieldExpression.equals("value")) { - return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); - } else { - throw new UnsupportedOperationException("Only KvCoder has fields."); + switch (fieldExpression) { + case "key": + return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); + case "value": + return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); + default: + throw new UnsupportedOperationException("Only KvCoder has fields."); } } @@ -151,12 +149,13 @@ public String[] getFieldNames() { @Override public int getFieldIndex(String fieldName) { - if (fieldName.equals("key")) { - return 0; - } else if (fieldName.equals("value")) { - return 1; - } else { - return -1; + switch (fieldName) { + case "key": + return 0; + case "value": + return 1; + default: + return -1; } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java index f4b6275963b0..2096e27d5d5e 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java @@ -17,9 +17,7 @@ */ package com.dataartisans.flink.dataflow.translation.types; -import com.google.cloud.dataflow.sdk.coders.VoidCoder; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.runtime.ValueSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java index d4f5ae2bd8b1..ff59db7e31bc 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java @@ -24,7 +24,7 @@ import com.google.common.base.Joiner; import org.apache.flink.test.util.JavaProgramTestBase; -import java.util.Arrays; +import java.util.Collections; import java.util.List; @@ -50,7 +50,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - List strings = Arrays.asList(); + List strings = Collections.emptyList(); Pipeline p = FlinkTestPipeline.create(); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java index 39e4a86cb7ee..cbf5d77744b6 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java @@ -51,8 +51,8 @@ public class JoinExamples { public static PCollection joinEvents(PCollection eventsTable, PCollection countryCodes) throws Exception { - final TupleTag eventInfoTag = new TupleTag(); - final TupleTag countryInfoTag = new TupleTag(); + final TupleTag eventInfoTag = new TupleTag<>(); + final TupleTag countryInfoTag = new TupleTag<>(); // transform both input collections to tuple collections, where the keys are country // codes in both cases. @@ -76,7 +76,7 @@ public void processElement(ProcessContext c) { KV e = c.element(); CoGbkResult val = e.getValue(); String countryCode = e.getKey(); - String countryName = "none"; + String countryName; countryName = e.getValue().getOnly(countryInfoTag); for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) { // Generate a string that combines information from both collection values @@ -87,7 +87,7 @@ public void processElement(ProcessContext c) { })); // write to GCS - PCollection formattedResults = finalResultCollection + return finalResultCollection .apply(ParDo.of(new DoFn, String>() { @Override public void processElement(ProcessContext c) { @@ -96,7 +96,6 @@ public void processElement(ProcessContext c) { c.output(outputstring); } })); - return formattedResults; } /** @@ -137,7 +136,7 @@ public void processElement(ProcessContext c) { *

* Inherits standard configuration options. */ - private static interface Options extends PipelineOptions { + private interface Options extends PipelineOptions { @Description("Path of the file to write to") @Validation.Required String getOutput(); From 4f139552ed88acafda616c53364b811004950d81 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 24 Dec 2015 11:28:14 +0100 Subject: [PATCH 118/149] make Avro type extraction more explicit --- .../FlinkTransformTranslators.java | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 10a77ec045e9..62232ce5a422 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -159,7 +159,6 @@ private static class AvroIOReadTranslator implements FlinkPipelineTranslator. private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslator.class); @Override - @SuppressWarnings("unchecked") public void translateNode(AvroIO.Read.Bound transform, TranslationContext context) { String path = transform.getFilepattern(); String name = transform.getName(); @@ -169,17 +168,21 @@ public void translateNode(AvroIO.Read.Bound transform, TranslationContext con TypeInformation typeInformation = context.getTypeInfo(output); // This is super hacky, but unfortunately we cannot get the type otherwise - Class avroType = null; + Class extractedAvroType; try { Field typeField = transform.getClass().getDeclaredField("type"); typeField.setAccessible(true); - avroType = (Class) typeField.get(transform); + @SuppressWarnings("unchecked") + Class avroType = (Class) typeField.get(transform); + extractedAvroType = avroType; } catch (NoSuchFieldException | IllegalAccessException e) { // we know that the field is there and it is accessible - System.out.println("Could not access type from AvroIO.Bound: " + e); + throw new RuntimeException("Could not access type from AvroIO.Bound", e); } - DataSource source = new DataSource<>(context.getExecutionEnvironment(), new AvroInputFormat<>(new Path(path), avroType), typeInformation, name); + DataSource source = new DataSource<>(context.getExecutionEnvironment(), + new AvroInputFormat<>(new Path(path), extractedAvroType), + typeInformation, name); context.setOutputDataSet(output, source); } @@ -189,7 +192,6 @@ private static class AvroIOWriteTranslator implements FlinkPipelineTranslator private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslator.class); @Override - @SuppressWarnings("unchecked") public void translateNode(AvroIO.Write.Bound transform, TranslationContext context) { DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); String filenamePrefix = transform.getFilenamePrefix(); @@ -203,18 +205,20 @@ public void translateNode(AvroIO.Write.Bound transform, TranslationContext co LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); // This is super hacky, but unfortunately we cannot get the type otherwise - Class avroType = null; + Class extractedAvroType; try { Field typeField = transform.getClass().getDeclaredField("type"); typeField.setAccessible(true); - avroType = (Class) typeField.get(transform); + @SuppressWarnings("unchecked") + Class avroType = (Class) typeField.get(transform); + extractedAvroType = avroType; } catch (NoSuchFieldException | IllegalAccessException e) { // we know that the field is there and it is accessible - System.out.println("Could not access type from AvroIO.Bound: " + e); + throw new RuntimeException("Could not access type from AvroIO.Bound", e); } DataSink dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path - (filenamePrefix), avroType)); + (filenamePrefix), extractedAvroType)); if (numShards > 0) { dataSink.setParallelism(numShards); From 03dfb1de1552ebb8f2d03df831708e2c2ad4036f Mon Sep 17 00:00:00 2001 From: Max Date: Fri, 15 Jan 2016 18:52:59 +0100 Subject: [PATCH 119/149] [maven] correct license and formatting --- runners/flink/pom.xml | 75 ++++++++++++++++++++++--------------------- 1 file changed, 38 insertions(+), 37 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 288de1be3ada..6102d742e9b4 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -1,18 +1,20 @@ @@ -25,7 +27,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans Flink Dataflow Runner jar - + 2015 @@ -39,13 +41,12 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans UTF-8 UTF-8 - 0.10.1 - - - com.dataartisans.flink.dataflow.examples.WordCount - kinglear.txt - wordcounts.txt - 1 + 0.10.1 + + com.dataartisans.flink.dataflow.examples.WordCount + kinglear.txt + wordcounts.txt + 1 @@ -114,7 +115,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans - + @@ -134,7 +135,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans - + org.apache.maven.plugins maven-compiler-plugin 3.1 @@ -164,7 +165,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans maven-surefire-plugin 2.17 - + org.apache.maven.plugins @@ -205,22 +206,22 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans - - org.codehaus.mojo - exec-maven-plugin - 1.2.1 - - java - - -classpath - - ${clazz} - --input=${input} - --output=${output} - --parallelism=${parallelism} - - - + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + java + + -classpath + + ${clazz} + --input=${input} + --output=${output} + --parallelism=${parallelism} + + + From 3947750d8ec8a68d366fbf820434b199676dec8f Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 18 Jan 2016 16:46:42 +0100 Subject: [PATCH 120/149] [tests] refactor ReadSourceITCase --- .../flink/dataflow/ReadSourceITCase.java | 122 +++++++++--------- 1 file changed, 62 insertions(+), 60 deletions(-) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java index c4dafaa6731f..ba675b1bb92f 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java @@ -75,87 +75,89 @@ public void processElement(ProcessContext c) throws Exception { result.apply(TextIO.Write.to(resultPath)); p.run(); } -} -class ReadSource extends BoundedSource { - final int from; - final int to; - ReadSource(int from, int to) { - this.from = from; - this.to = to; - } + private static class ReadSource extends BoundedSource { + final int from; + final int to; - @Override - public List splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options) - throws Exception { - List res = new ArrayList<>(); - FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class); - int numWorkers = flinkOptions.getParallelism(); - Preconditions.checkArgument(numWorkers > 0, "Number of workers should be larger than 0."); - - float step = 1.0f * (to - from) / numWorkers; - for (int i = 0; i < numWorkers; ++i) { - res.add(new ReadSource(Math.round(from + i * step), Math.round(from + (i + 1) * step))); + ReadSource(int from, int to) { + this.from = from; + this.to = to; } - return res; - } - - @Override - public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { - return 8 * (to - from); - } - - @Override - public boolean producesSortedKeys(PipelineOptions options) throws Exception { - return true; - } - - @Override - public BoundedReader createReader(PipelineOptions options) throws IOException { - return new RangeReader(this); - } - - @Override - public void validate() {} - - @Override - public Coder getDefaultOutputCoder() { - return BigEndianIntegerCoder.of(); - } - private class RangeReader extends BoundedReader { - private int current; + @Override + public List splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options) + throws Exception { + List res = new ArrayList<>(); + FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class); + int numWorkers = flinkOptions.getParallelism(); + Preconditions.checkArgument(numWorkers > 0, "Number of workers should be larger than 0."); + + float step = 1.0f * (to - from) / numWorkers; + for (int i = 0; i < numWorkers; ++i) { + res.add(new ReadSource(Math.round(from + i * step), Math.round(from + (i + 1) * step))); + } + return res; + } - public RangeReader(ReadSource source) { - this.current = source.from - 1; + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return 8 * (to - from); } @Override - public boolean start() throws IOException { + public boolean producesSortedKeys(PipelineOptions options) throws Exception { return true; } @Override - public boolean advance() throws IOException { - current++; - return (current < to); + public BoundedReader createReader(PipelineOptions options) throws IOException { + return new RangeReader(this); } @Override - public Integer getCurrent() { - return current; - } + public void validate() {} @Override - public void close() throws IOException { - // Nothing + public Coder getDefaultOutputCoder() { + return BigEndianIntegerCoder.of(); } - @Override - public BoundedSource getCurrentSource() { - return ReadSource.this; + private class RangeReader extends BoundedReader { + private int current; + + public RangeReader(ReadSource source) { + this.current = source.from - 1; + } + + @Override + public boolean start() throws IOException { + return true; + } + + @Override + public boolean advance() throws IOException { + current++; + return (current < to); + } + + @Override + public Integer getCurrent() { + return current; + } + + @Override + public void close() throws IOException { + // Nothing + } + + @Override + public BoundedSource getCurrentSource() { + return ReadSource.this; + } } } } + From 49d295b7d72d32da2e652a6f8019e3fb3c7819d1 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 18 Jan 2016 18:46:34 +0100 Subject: [PATCH 121/149] [runner] add translator for Write.Bound for custom sinks --- .../FlinkTransformTranslators.java | 16 +- .../wrappers/SinkOutputFormat.java | 118 +++++++++++++ .../flink/dataflow/WriteSinkITCase.java | 156 ++++++++++++++++++ 3 files changed, 289 insertions(+), 1 deletion(-) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java index 62232ce5a422..c1d78c02eedf 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java @@ -29,6 +29,7 @@ import com.dataartisans.flink.dataflow.translation.functions.UnionCoder; import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation; +import com.dataartisans.flink.dataflow.translation.wrappers.SinkOutputFormat; import com.dataartisans.flink.dataflow.translation.wrappers.SourceInputFormat; import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException; @@ -46,6 +47,7 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.transforms.Write; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema; import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; @@ -123,7 +125,7 @@ public class FlinkTransformTranslators { TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslator()); TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslator()); -// TRANSLATORS.put(Write.Bound.class, new ReadSourceTranslator()); + TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslator()); TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslator()); TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslator()); @@ -288,6 +290,18 @@ public void translateNode(ConsoleIO.Write.Bound transform, TranslationContext co } } + private static class WriteSinkTranslator implements FlinkPipelineTranslator.TransformTranslator> { + + @Override + public void translateNode(Write.Bound transform, TranslationContext context) { + String name = transform.getName(); + PValue input = context.getInput(transform); + DataSet inputDataSet = context.getInputDataSet(input); + + inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name); + } + } + private static class GroupByKeyOnlyTranslator implements FlinkPipelineTranslator.TransformTranslator> { @Override diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java new file mode 100644 index 000000000000..d87b24031a79 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java @@ -0,0 +1,118 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.cloud.dataflow.sdk.io.Sink; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import com.google.cloud.dataflow.sdk.transforms.Write; +import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.configuration.Configuration; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.lang.reflect.Field; + +/** + * Wrapper class to use generic Write.Bound transforms as sinks. + * @param The type of the incoming records. + */ +public class SinkOutputFormat implements OutputFormat { + + private final Sink sink; + + private transient PipelineOptions pipelineOptions; + + private Sink.WriteOperation writeOperation; + private Sink.Writer writer; + + public SinkOutputFormat(Write.Bound transform, PipelineOptions pipelineOptions) { + this.sink = extractSink(transform); + this.pipelineOptions = Preconditions.checkNotNull(pipelineOptions); + } + + private Sink extractSink(Write.Bound transform) { + // TODO possibly add a getter in the upstream + try { + Field sinkField = transform.getClass().getDeclaredField("sink"); + sinkField.setAccessible(true); + @SuppressWarnings("unchecked") + Sink extractedSink = (Sink) sinkField.get(transform); + return extractedSink; + } catch (NoSuchFieldException | IllegalAccessException e) { + throw new RuntimeException("Could not acquire custom sink field.", e); + } + } + + @Override + public void configure(Configuration configuration) { + writeOperation = sink.createWriteOperation(pipelineOptions); + try { + writeOperation.initialize(pipelineOptions); + } catch (Exception e) { + throw new RuntimeException("Failed to initialize the write operation.", e); + } + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + try { + writer = writeOperation.createWriter(pipelineOptions); + } catch (Exception e) { + throw new IOException("Couldn't create writer.", e); + } + try { + writer.open(String.valueOf(taskNumber)); + } catch (Exception e) { + throw new IOException("Couldn't open writer.", e); + } + } + + @Override + public void writeRecord(T record) throws IOException { + try { + writer.write(record); + } catch (Exception e) { + throw new IOException("Couldn't write record.", e); + } + } + + @Override + public void close() throws IOException { + try { + writer.close(); + } catch (Exception e) { + throw new IOException("Couldn't close writer.", e); + } + } + + private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException { + out.defaultWriteObject(); + ObjectMapper mapper = new ObjectMapper(); + mapper.writeValue(out, pipelineOptions); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + ObjectMapper mapper = new ObjectMapper(); + pipelineOptions = mapper.readValue(in, PipelineOptions.class); + } +} diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java new file mode 100644 index 000000000000..c8302e89c630 --- /dev/null +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java @@ -0,0 +1,156 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.io.Sink; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.Write; +import com.google.common.base.Joiner; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.test.util.JavaProgramTestBase; + +import java.io.File; +import java.io.PrintWriter; +import java.net.URI; + +import static junit.framework.Assert.*; + +/** + * Tests the translation of custom Write.Bound sinks. + */ +public class WriteSinkITCase extends JavaProgramTestBase { + + protected String resultPath; + + public WriteSinkITCase(){ + } + + static final String[] EXPECTED_RESULT = new String[] { + "Joe red 3", "Mary blue 4", "Max yellow 23"}; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } + + @Override + protected void testProgram() throws Exception { + runProgram(resultPath); + } + + private static void runProgram(String resultPath) { + Pipeline p = FlinkTestPipeline.create(); + + p.apply(Create.of(EXPECTED_RESULT)).setCoder(StringUtf8Coder.of()) + .apply("CustomSink", Write.to(new MyCustomSink(resultPath))); + + p.run(); + } + + /** + * Simple custom sink which writes to a file. + */ + private static class MyCustomSink extends Sink { + + private final String resultPath; + + public MyCustomSink(String resultPath) { + this.resultPath = resultPath; + } + + @Override + public void validate(PipelineOptions options) { + assertNotNull(options); + } + + @Override + public WriteOperation createWriteOperation(PipelineOptions options) { + return new MyWriteOperation(); + } + + private class MyWriteOperation extends WriteOperation { + + @Override + public Coder getWriterResultCoder() { + return StringUtf8Coder.of(); + } + + @Override + public void initialize(PipelineOptions options) throws Exception { + + } + + @Override + public void finalize(Iterable writerResults, PipelineOptions options) throws Exception { + + } + + @Override + public Writer createWriter(PipelineOptions options) throws Exception { + return new MyWriter(); + } + + @Override + public Sink getSink() { + return MyCustomSink.this; + } + + /** + * Simple Writer which writes to a file. + */ + private class MyWriter extends Writer { + + private PrintWriter internalWriter; + + @Override + public void open(String uId) throws Exception { + Path path = new Path(resultPath + "/" + uId); + FileSystem.get(new URI("file:///")).create(path, false); + internalWriter = new PrintWriter(new File(path.toUri())); + } + + @Override + public void write(String value) throws Exception { + internalWriter.println(value); + } + + @Override + public String close() throws Exception { + internalWriter.close(); + return resultPath; + } + + @Override + public WriteOperation getWriteOperation() { + return MyWriteOperation.this; + } + } + } + } + +} + From 0d68bb220c7c7e7a364efe48947518c3deacbe61 Mon Sep 17 00:00:00 2001 From: Max Date: Tue, 19 Jan 2016 10:38:49 +0100 Subject: [PATCH 122/149] [sink] generate unique id for writer initialization --- .../dataflow/translation/wrappers/SinkOutputFormat.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java index d87b24031a79..b10c86f9c907 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java @@ -23,13 +23,16 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; import com.google.cloud.dataflow.sdk.transforms.Write; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.AbstractID; import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.lang.reflect.Field; +import java.util.UUID; /** * Wrapper class to use generic Write.Bound transforms as sinks. @@ -44,6 +47,8 @@ public class SinkOutputFormat implements OutputFormat { private Sink.WriteOperation writeOperation; private Sink.Writer writer; + private AbstractID uid = new AbstractID(); + public SinkOutputFormat(Write.Bound transform, PipelineOptions pipelineOptions) { this.sink = extractSink(transform); this.pipelineOptions = Preconditions.checkNotNull(pipelineOptions); @@ -80,7 +85,7 @@ public void open(int taskNumber, int numTasks) throws IOException { throw new IOException("Couldn't create writer.", e); } try { - writer.open(String.valueOf(taskNumber)); + writer.open(uid + "-" + String.valueOf(taskNumber)); } catch (Exception e) { throw new IOException("Couldn't open writer.", e); } From 9d9ccf59681c8fdbab4d26c5e37b3c9f2ad6371d Mon Sep 17 00:00:00 2001 From: Max Date: Tue, 19 Jan 2016 14:30:26 +0100 Subject: [PATCH 123/149] [readme] add a section on how to submit cluster programs --- runners/flink/README.md | 81 ++++++++++++++++++++++++++++++++++++++--- 1 file changed, 76 insertions(+), 5 deletions(-) diff --git a/runners/flink/README.md b/runners/flink/README.md index a4b1d7eb69ec..e8c2170560dc 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -60,11 +60,82 @@ Congratulations, you have run your first Google Dataflow program on top of Apach # Running Dataflow on Flink on a cluster -You can run your Dataflow program on a Apache Flink cluster as well. For more -information, please visit the [Apache Flink Website](http://flink.apache.org) or -contact the -[Mailinglists](http://flink.apache.org/community.html#mailing-lists). +You can run your Dataflow program on an Apache Flink cluster. Please start off by creating a new +Maven project. + + mvn archetype:generate -DgroupId=com.mycompany.dataflow -DartifactId=dataflow-test \ + -DarchetypeArtifactId=maven-archetype-quickstart -DinteractiveMode=false + +The contents of the root `pom.xml` should be slightly changed aftewards (explanation below): + +```xml + + + 4.0.0 + + com.mycompany.dataflow + dataflow-test + 1.0 + + + + com.dataartisans + flink-dataflow + 0.2 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 2.4.1 + + + package + + shade + + + + + WordCount + + + + + org.apache.flink:* + + + + + + + + + + + + +``` + +The following changes have been made: + +1. The Flink Dataflow Runner was added as a dependency. + +2. The Maven Shade plugin was added to build a fat jar. + +A fat jar is necessary if you want to submit your Dataflow code to a Flink cluster. The fat jar +includes your program code but also Dataflow code which is necessary during runtime. Note that this +step is necessary because the Dataflow Runner is not part of Flink. + +For more information, please visit the [Apache Flink Website](http://flink.apache.org) or contact +the [Mailinglists](http://flink.apache.org/community.html#mailing-lists). # Streaming -Streaming support is currently under development. See the `streaming` branch for the current version. +Streaming support is currently under development. See the `streaming_new` branch for the current +work in progress version. From aac96d77498669f4bf66d9882737d8a59b4d4c27 Mon Sep 17 00:00:00 2001 From: Max Date: Tue, 19 Jan 2016 14:33:55 +0100 Subject: [PATCH 124/149] [readme] add hint on how to submit jar to cluster --- runners/flink/README.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/runners/flink/README.md b/runners/flink/README.md index e8c2170560dc..8c06c1da3414 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -132,6 +132,11 @@ A fat jar is necessary if you want to submit your Dataflow code to a Flink clust includes your program code but also Dataflow code which is necessary during runtime. Note that this step is necessary because the Dataflow Runner is not part of Flink. +You can then build the jar using `mvn clean package`. Please submit the fat jar in the `target` +folder to the Flink cluster using the command-line utility like so: + + ./bin/flink run /path/to/fat.jar + For more information, please visit the [Apache Flink Website](http://flink.apache.org) or contact the [Mailinglists](http://flink.apache.org/community.html#mailing-lists). From d4a651aea533f99f52a794878da79416f7309d65 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 9 Dec 2015 17:30:53 +0100 Subject: [PATCH 125/149] [runner] add streaming support with checkpointing --- runners/flink/pom.xml | 28 + .../FlinkJobExecutionEnvironment.java | 238 +++++++ .../flink/dataflow/FlinkPipelineRunner.java | 99 +-- .../flink/dataflow/examples/WordCount.java | 2 +- .../examples/streaming/AutoComplete.java | 384 +++++++++++ .../examples/streaming/JoinExamples.java | 157 +++++ .../KafkaWindowedWordCountExample.java | 138 ++++ .../examples/streaming/WindowedWordCount.java | 126 ++++ .../FlinkBatchPipelineTranslator.java | 152 +++++ ...va => FlinkBatchTransformTranslators.java} | 123 ++-- ...java => FlinkBatchTranslationContext.java} | 4 +- .../translation/FlinkPipelineTranslator.java | 145 +---- .../FlinkStreamingPipelineTranslator.java | 138 ++++ .../FlinkStreamingTransformTranslators.java | 356 +++++++++++ .../FlinkStreamingTranslationContext.java | 86 +++ ...erComperator.java => CoderComparator.java} | 10 +- .../types/CoderTypeInformation.java | 6 +- .../types/CoderTypeSerializer.java | 2 - .../translation/types/KvCoderComperator.java | 2 +- .../types/VoidCoderTypeSerializer.java | 1 - .../wrappers/SourceInputFormat.java | 4 +- .../streaming/FlinkAbstractParDoWrapper.java | 274 ++++++++ .../FlinkGroupAlsoByWindowWrapper.java | 601 ++++++++++++++++++ .../streaming/FlinkGroupByKeyWrapper.java | 56 ++ .../FlinkParDoBoundMultiWrapper.java | 72 +++ .../streaming/FlinkParDoBoundWrapper.java | 89 +++ .../streaming/io/UnboundedFlinkSource.java | 76 +++ .../streaming/io/UnboundedSocketSource.java | 228 +++++++ .../streaming/io/UnboundedSourceWrapper.java | 120 ++++ .../state/AbstractFlinkTimerInternals.java | 139 ++++ .../streaming/state/FlinkStateInternals.java | 533 ++++++++++++++++ .../state/StateCheckpointReader.java | 89 +++ .../streaming/state/StateCheckpointUtils.java | 152 +++++ .../state/StateCheckpointWriter.java | 127 ++++ .../wrappers/streaming/state/StateType.java | 67 ++ .../streaming/GroupAlsoByWindowTest.java | 507 +++++++++++++++ .../streaming/StateSerializationTest.java | 257 ++++++++ .../flink/dataflow/util/JoinExamples.java | 4 +- 38 files changed, 5286 insertions(+), 306 deletions(-) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java rename runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/{FlinkTransformTranslators.java => FlinkBatchTransformTranslators.java} (82%) rename runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/{TranslationContext.java => FlinkBatchTranslationContext.java} (96%) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java rename runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/{CoderComperator.java => CoderComparator.java} (97%) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 6102d742e9b4..14693b8d0322 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -69,6 +69,18 @@ flink-core ${flink.version} + + org.apache.flink + flink-streaming-java + ${flink.version} + + + org.apache.flink + flink-streaming-java + ${flink.version} + test + test-jar + org.apache.flink flink-java @@ -114,6 +126,22 @@ + + org.apache.flink + flink-connector-kafka + ${flink.version} + + + org.mockito + mockito-all + 1.9.5 + test + + + org.apache.flink + flink-streaming-java + ${flink.version} + diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java new file mode 100644 index 000000000000..66d60fa5a9b6 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java @@ -0,0 +1,238 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow; + +import com.dataartisans.flink.dataflow.translation.FlinkPipelineTranslator; +import com.dataartisans.flink.dataflow.translation.FlinkBatchPipelineTranslator; +import com.dataartisans.flink.dataflow.translation.FlinkStreamingPipelineTranslator; +import com.google.cloud.dataflow.sdk.Pipeline; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.java.CollectionEnvironment; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.environment.RemoteStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +public class FlinkJobExecutionEnvironment { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkJobExecutionEnvironment.class); + + private final FlinkPipelineOptions options; + + /** + * The Flink Batch execution environment. This is instantiated to either a + * {@link org.apache.flink.api.java.CollectionEnvironment}, + * a {@link org.apache.flink.api.java.LocalEnvironment} or + * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration + * options. + */ + private ExecutionEnvironment flinkBatchEnv; + + + /** + * The Flink Streaming execution environment. This is instantiated to either a + * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or + * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending + * on the configuration options, and more specifically, the url of the master url. + */ + private StreamExecutionEnvironment flinkStreamEnv; + + /** + * Translator for this FlinkPipelineRunner. Its role is to translate the Dataflow operators to + * their Flink based counterparts. Based on the options provided by the user, if we have a streaming job, + * this is instantiated to a FlinkStreamingPipelineTranslator. In other case, i.e. a batch job, + * a FlinkBatchPipelineTranslator is created. + */ + private FlinkPipelineTranslator flinkPipelineTranslator; + + public FlinkJobExecutionEnvironment(FlinkPipelineOptions options) { + if (options == null) { + throw new IllegalArgumentException("Options in the FlinkJobExecutionEnvironment cannot be NULL."); + } + this.options = options; + this.createJobEnvironment(); + this.createJobGraphTranslator(); + } + + /** + * Depending on the type of job (Streaming or Batch) and the user-specified options, + * this method creates the adequate ExecutionEnvironment. + */ + private void createJobEnvironment() { + if (options.isStreaming()) { + LOG.info("Creating the required STREAMING Environment."); + createStreamExecutionEnvironment(); + } else { + LOG.info("Creating the required BATCH Environment."); + createBatchExecutionEnvironment(); + } + } + + /** + * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph + * translator. In the case of batch, it will work with DataSets, while for streaming, it will work + * with DataStreams. + */ + private void createJobGraphTranslator() { + checkInitializationState(); + if (this.flinkPipelineTranslator != null) { + throw new IllegalStateException("JobGraphTranslator already initialized."); + } + + this.flinkPipelineTranslator = options.isStreaming() ? + new FlinkStreamingPipelineTranslator(flinkStreamEnv, options) : + new FlinkBatchPipelineTranslator(flinkBatchEnv, options); + } + + public void translate(Pipeline pipeline) { + checkInitializationState(); + if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) { + createJobEnvironment(); + } + if (this.flinkPipelineTranslator == null) { + createJobGraphTranslator(); + } + this.flinkPipelineTranslator.translate(pipeline); + } + + public JobExecutionResult executeJob() throws Exception { + if (options.isStreaming()) { + + System.out.println("Plan: " + this.flinkStreamEnv.getExecutionPlan()); + + if (this.flinkStreamEnv == null) { + throw new RuntimeException("JobExecutionEnvironment not initialized."); + } + if (this.flinkPipelineTranslator == null) { + throw new RuntimeException("JobGraphTranslator not initialized."); + } + return this.flinkStreamEnv.execute(); + } else { + if (this.flinkBatchEnv == null) { + throw new RuntimeException("JobExecutionEnvironment not initialized."); + } + if (this.flinkPipelineTranslator == null) { + throw new RuntimeException("JobGraphTranslator not initialized."); + } + return this.flinkBatchEnv.execute(); + } + } + + /** + * If the submitted job is a batch processing job, this method creates the adequate + * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending + * on the user-specified options. + */ + private void createBatchExecutionEnvironment() { + if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) { + throw new RuntimeException("JobExecutionEnvironment already initialized."); + } + + String masterUrl = options.getFlinkMaster(); + this.flinkStreamEnv = null; + + // depending on the master, create the right environment. + if (masterUrl.equals("[local]")) { + this.flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment(); + } else if (masterUrl.equals("[collection]")) { + this.flinkBatchEnv = new CollectionEnvironment(); + } else if (masterUrl.equals("[auto]")) { + this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment(); + } else if (masterUrl.matches(".*:\\d*")) { + String[] parts = masterUrl.split(":"); + List stagingFiles = options.getFilesToStage(); + this.flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0], + Integer.parseInt(parts[1]), + stagingFiles.toArray(new String[stagingFiles.size()])); + } else { + LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl); + this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment(); + } + + // set the correct parallelism. + if (options.getParallelism() != -1 && !(this.flinkBatchEnv instanceof CollectionEnvironment)) { + this.flinkBatchEnv.setParallelism(options.getParallelism()); + } + + // set parallelism in the options (required by some execution code) + options.setParallelism(flinkBatchEnv.getParallelism()); + } + + /** + * If the submitted job is a stream processing job, this method creates the adequate + * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending + * on the user-specified options. + */ + private void createStreamExecutionEnvironment() { + if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) { + throw new RuntimeException("JobExecutionEnvironment already initialized."); + } + + String masterUrl = options.getFlinkMaster(); + this.flinkBatchEnv = null; + + // depending on the master, create the right environment. + if (masterUrl.equals("[local]")) { + this.flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment(); + } else if (masterUrl.equals("[auto]")) { + this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + } else if (masterUrl.matches(".*:\\d*")) { + String[] parts = masterUrl.split(":"); + List stagingFiles = options.getFilesToStage(); + this.flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0], + Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()])); + } else { + LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl); + this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + } + + // set the correct parallelism. + if (options.getParallelism() != -1) { + this.flinkStreamEnv.setParallelism(options.getParallelism()); + } + + // set parallelism in the options (required by some execution code) + options.setParallelism(flinkStreamEnv.getParallelism()); + + // although we do not use the generated timestamps, + // enabling timestamps is needed for the watermarks. + this.flinkStreamEnv.getConfig().enableTimestamps(); + + this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); + + this.flinkStreamEnv.enableCheckpointing(1000); + this.flinkStreamEnv.setNumberOfExecutionRetries(5); + + LOG.info("Setting execution retry delay to 3 sec"); + this.flinkStreamEnv.getConfig().setExecutionRetryDelay(3000); + } + + private final void checkInitializationState() { + if (this.options == null) { + throw new IllegalStateException("FlinkJobExecutionEnvironment is not initialized yet."); + } + + if (options.isStreaming() && this.flinkBatchEnv != null) { + throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment."); + } else if (!options.isStreaming() && this.flinkStreamEnv != null) { + throw new IllegalStateException("Attempted to run a Batch Job with a Streaming Execution Environment."); + } + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java index ae31f48fc3ac..f57fed2dae20 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java @@ -15,7 +15,6 @@ */ package com.dataartisans.flink.dataflow; -import com.dataartisans.flink.dataflow.translation.FlinkPipelineTranslator; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; @@ -28,8 +27,6 @@ import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.java.CollectionEnvironment; -import org.apache.flink.api.java.ExecutionEnvironment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,27 +42,19 @@ * A {@link PipelineRunner} that executes the operations in the * pipeline by first translating them to a Flink Plan and then executing them either locally * or on a Flink cluster, depending on the configuration. - * + *

* This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}. */ public class FlinkPipelineRunner extends PipelineRunner { private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineRunner.class); - /** Provided options. */ - private final FlinkPipelineOptions options; - /** - * The Flink execution environment. This is instantiated to either a - * {@link org.apache.flink.api.java.CollectionEnvironment}, - * a {@link org.apache.flink.api.java.LocalEnvironment} or - * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration - * options. + * Provided options. */ - private final ExecutionEnvironment flinkEnv; + private final FlinkPipelineOptions options; - /** Translator for this FlinkPipelineRunner, based on options. */ - private final FlinkPipelineTranslator translator; + private final FlinkJobExecutionEnvironment flinkJobEnv; /** * Construct a runner from the provided options. @@ -109,90 +98,38 @@ public static FlinkPipelineRunner fromOptions(PipelineOptions options) { flinkOptions.setFlinkMaster("[auto]"); } - if (flinkOptions.isStreaming()) { - throw new RuntimeException("Streaming is currently not supported."); - } - return new FlinkPipelineRunner(flinkOptions); } private FlinkPipelineRunner(FlinkPipelineOptions options) { this.options = options; - this.flinkEnv = createExecutionEnvironment(options); - - // set parallelism in the options (required by some execution code) - options.setParallelism(flinkEnv.getParallelism()); - - this.translator = new FlinkPipelineTranslator(flinkEnv, options); - } - - /** - * Create Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending - * on the options. - */ - private ExecutionEnvironment createExecutionEnvironment(FlinkPipelineOptions options) { - String masterUrl = options.getFlinkMaster(); - - - if (masterUrl.equals("[local]")) { - ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - if (options.getParallelism() != -1) { - env.setParallelism(options.getParallelism()); - } - return env; - } else if (masterUrl.equals("[collection]")) { - return new CollectionEnvironment(); - } else if (masterUrl.equals("[auto]")) { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - if (options.getParallelism() != -1) { - env.setParallelism(options.getParallelism()); - } - return env; - } else if (masterUrl.matches(".*:\\d*")) { - String[] parts = masterUrl.split(":"); - List stagingFiles = options.getFilesToStage(); - ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(parts[0], - Integer.parseInt(parts[1]), - stagingFiles.toArray(new String[stagingFiles.size()])); - if (options.getParallelism() != -1) { - env.setParallelism(options.getParallelism()); - } - return env; - } else { - LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl); - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - if (options.getParallelism() != -1) { - env.setParallelism(options.getParallelism()); - } - return env; - } + this.flinkJobEnv = new FlinkJobExecutionEnvironment(options); } @Override public FlinkRunnerResult run(Pipeline pipeline) { LOG.info("Executing pipeline using FlinkPipelineRunner."); - + LOG.info("Translating pipeline to Flink program."); - - translator.translate(pipeline); - + + this.flinkJobEnv.translate(pipeline); + LOG.info("Starting execution of Flink program."); JobExecutionResult result; try { - result = flinkEnv.execute(); - } - catch (Exception e) { + result = this.flinkJobEnv.executeJob(); + } catch (Exception e) { LOG.error("Pipeline execution failed", e); throw new RuntimeException("Pipeline execution failed", e); } - + LOG.info("Execution finished in {} msecs", result.getNetRuntime()); - + Map accumulators = result.getAllAccumulatorResults(); if (accumulators != null && !accumulators.isEmpty()) { LOG.info("Final aggregator values:"); - + for (Map.Entry entry : result.getAllAccumulatorResults().entrySet()) { LOG.info("{} : {}", entry.getKey(), entry.getValue()); } @@ -230,16 +167,18 @@ public Output apply( ///////////////////////////////////////////////////////////////////////////// @Override - public String toString() { return "DataflowPipelineRunner#" + hashCode(); } + public String toString() { + return "DataflowPipelineRunner#" + hashCode(); + } /** * Attempts to detect all the resources the class loader has access to. This does not recurse * to class loader parents stopping it from pulling in resources from the system class loader. * * @param classLoader The URLClassLoader to use to detect resources to stage. - * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one - * of the resources the class loader exposes is not a file resource. * @return A list of absolute paths to the resources the class loader uses. + * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one + * of the resources the class loader exposes is not a file resource. */ protected static List detectClassPathResourcesToStage(ClassLoader classLoader) { if (!(classLoader instanceof URLClassLoader)) { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java index 82f1e46302c2..78577783beb5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java @@ -43,7 +43,7 @@ public interface Options extends PipelineOptions, FlinkPipelineOptions { String getOutput(); void setOutput(String value); } - + public static void main(String[] args) { Options options = PipelineOptionsFactory.fromArgs(args).withValidation() diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java new file mode 100644 index 000000000000..0245a7b21d15 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java @@ -0,0 +1,384 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.dataartisans.flink.dataflow.examples.streaming; + +import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSocketSource; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.io.*; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.*; +import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.*; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PBegin; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PCollectionList; +import org.joda.time.Duration; + +import java.io.IOException; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * To run the example, first open a socket on a terminal by executing the command: + *

  • + *
  • + * nc -lk 9999 + *
  • + * + * and then launch the example. Now whatever you type in the terminal is going to be + * the input to the program. + * */ +public class AutoComplete { + + /** + * A PTransform that takes as input a list of tokens and returns + * the most common tokens per prefix. + */ + public static class ComputeTopCompletions + extends PTransform, PCollection>>> { + private static final long serialVersionUID = 0; + + private final int candidatesPerPrefix; + private final boolean recursive; + + protected ComputeTopCompletions(int candidatesPerPrefix, boolean recursive) { + this.candidatesPerPrefix = candidatesPerPrefix; + this.recursive = recursive; + } + + public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursive) { + return new ComputeTopCompletions(candidatesPerPrefix, recursive); + } + + @Override + public PCollection>> apply(PCollection input) { + PCollection candidates = input + // First count how often each token appears. + .apply(new Count.PerElement()) + + // Map the KV outputs of Count into our own CompletionCandiate class. + .apply(ParDo.named("CreateCompletionCandidates").of( + new DoFn, CompletionCandidate>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + CompletionCandidate cand = new CompletionCandidate(c.element().getKey(), c.element().getValue()); + c.output(cand); + } + })); + + // Compute the top via either a flat or recursive algorithm. + if (recursive) { + return candidates + .apply(new ComputeTopRecursive(candidatesPerPrefix, 1)) + .apply(Flatten.>>pCollections()); + } else { + return candidates + .apply(new ComputeTopFlat(candidatesPerPrefix, 1)); + } + } + } + + /** + * Lower latency, but more expensive. + */ + private static class ComputeTopFlat + extends PTransform, + PCollection>>> { + private static final long serialVersionUID = 0; + + private final int candidatesPerPrefix; + private final int minPrefix; + + public ComputeTopFlat(int candidatesPerPrefix, int minPrefix) { + this.candidatesPerPrefix = candidatesPerPrefix; + this.minPrefix = minPrefix; + } + + @Override + public PCollection>> apply( + PCollection input) { + return input + // For each completion candidate, map it to all prefixes. + .apply(ParDo.of(new AllPrefixes(minPrefix))) + + // Find and return the top candiates for each prefix. + .apply(Top.largestPerKey(candidatesPerPrefix) + .withHotKeyFanout(new HotKeyFanout())); + } + + private static class HotKeyFanout implements SerializableFunction { + private static final long serialVersionUID = 0; + + @Override + public Integer apply(String input) { + return (int) Math.pow(4, 5 - input.length()); + } + } + } + + /** + * Cheaper but higher latency. + * + *

    Returns two PCollections, the first is top prefixes of size greater + * than minPrefix, and the second is top prefixes of size exactly + * minPrefix. + */ + private static class ComputeTopRecursive + extends PTransform, + PCollectionList>>> { + private static final long serialVersionUID = 0; + + private final int candidatesPerPrefix; + private final int minPrefix; + + public ComputeTopRecursive(int candidatesPerPrefix, int minPrefix) { + this.candidatesPerPrefix = candidatesPerPrefix; + this.minPrefix = minPrefix; + } + + private class KeySizePartitionFn implements PartitionFn>> { + private static final long serialVersionUID = 0; + + @Override + public int partitionFor(KV> elem, int numPartitions) { + return elem.getKey().length() > minPrefix ? 0 : 1; + } + } + + private static class FlattenTops + extends DoFn>, CompletionCandidate> { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + for (CompletionCandidate cc : c.element().getValue()) { + c.output(cc); + } + } + } + + @Override + public PCollectionList>> apply( + PCollection input) { + if (minPrefix > 10) { + // Base case, partitioning to return the output in the expected format. + return input + .apply(new ComputeTopFlat(candidatesPerPrefix, minPrefix)) + .apply(Partition.of(2, new KeySizePartitionFn())); + } else { + // If a candidate is in the top N for prefix a...b, it must also be in the top + // N for a...bX for every X, which is typlically a much smaller set to consider. + // First, compute the top candidate for prefixes of size at least minPrefix + 1. + PCollectionList>> larger = input + .apply(new ComputeTopRecursive(candidatesPerPrefix, minPrefix + 1)); + // Consider the top candidates for each prefix of length minPrefix + 1... + PCollection>> small = + PCollectionList + .of(larger.get(1).apply(ParDo.of(new FlattenTops()))) + // ...together with those (previously excluded) candidates of length + // exactly minPrefix... + .and(input.apply(Filter.by(new SerializableFunction() { + private static final long serialVersionUID = 0; + + @Override + public Boolean apply(CompletionCandidate c) { + return c.getValue().length() == minPrefix; + } + }))) + .apply("FlattenSmall", Flatten.pCollections()) + // ...set the key to be the minPrefix-length prefix... + .apply(ParDo.of(new AllPrefixes(minPrefix, minPrefix))) + // ...and (re)apply the Top operator to all of them together. + .apply(Top.largestPerKey(candidatesPerPrefix)); + + PCollection>> flattenLarger = larger + .apply("FlattenLarge", Flatten.>>pCollections()); + + return PCollectionList.of(flattenLarger).and(small); + } + } + } + + /** + * A DoFn that keys each candidate by all its prefixes. + */ + private static class AllPrefixes + extends DoFn> { + private static final long serialVersionUID = 0; + + private final int minPrefix; + private final int maxPrefix; + public AllPrefixes(int minPrefix) { + this(minPrefix, Integer.MAX_VALUE); + } + public AllPrefixes(int minPrefix, int maxPrefix) { + this.minPrefix = minPrefix; + this.maxPrefix = maxPrefix; + } + @Override + public void processElement(ProcessContext c) { + String word = c.element().value; + for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) { + KV kv = KV.of(word.substring(0, i), c.element()); + c.output(kv); + } + } + } + + /** + * Class used to store tag-count pairs. + */ + @DefaultCoder(AvroCoder.class) + static class CompletionCandidate implements Comparable { + private long count; + private String value; + + public CompletionCandidate(String value, long count) { + this.value = value; + this.count = count; + } + + public String getValue() { + return value; + } + + // Empty constructor required for Avro decoding. + @SuppressWarnings("unused") + public CompletionCandidate() {} + + @Override + public int compareTo(CompletionCandidate o) { + if (this.count < o.count) { + return -1; + } else if (this.count == o.count) { + return this.value.compareTo(o.value); + } else { + return 1; + } + } + + @Override + public boolean equals(Object other) { + if (other instanceof CompletionCandidate) { + CompletionCandidate that = (CompletionCandidate) other; + return this.count == that.count && this.value.equals(that.value); + } else { + return false; + } + } + + @Override + public int hashCode() { + return Long.valueOf(count).hashCode() ^ value.hashCode(); + } + + @Override + public String toString() { + return "CompletionCandidate[" + value + ", " + count + "]"; + } + } + + static class ExtractWordsFn extends DoFn { + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + if (c.element().trim().isEmpty()) { + emptyLines.addValue(1L); + } + + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + /** + * Takes as input a the top candidates per prefix, and emits an entity + * suitable for writing to Datastore. + */ + static class FormatForPerTaskLocalFile extends DoFn>, String> { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + StringBuilder str = new StringBuilder(); + KV> elem = c.element(); + + str.append(elem.getKey() +" @ "+ c.window() +" -> "); + for(CompletionCandidate cand: elem.getValue()) { + str.append(cand.toString() + " "); + } + System.out.println(str.toString()); + c.output(str.toString()); + } + } + + /** + * Options supported by this class. + * + *

    Inherits standard Dataflow configuration options. + */ + private static interface Options extends WindowedWordCount.StreamingWordCountOptions { + @Description("Whether to use the recursive algorithm") + @Default.Boolean(true) + Boolean getRecursive(); + void setRecursive(Boolean value); + } + + public static void main(String[] args) throws IOException { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + options.setStreaming(true); + options.setRunner(FlinkPipelineRunner.class); + + PTransform> readSource = + Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("WordStream"); + WindowFn windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize())); + + // Create the pipeline. + Pipeline p = Pipeline.create(options); + PCollection>> toWrite = p + .apply(readSource) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Window.into(windowFn) + .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()) + .apply(ComputeTopCompletions.top(10, options.getRecursive())); + + toWrite + .apply(ParDo.named("FormatForPerTaskFile").of(new FormatForPerTaskLocalFile())) + .apply(TextIO.Write.to("./outputAutoComplete.txt")); + + p.run(); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java new file mode 100644 index 000000000000..b0cc4fa1fc42 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java @@ -0,0 +1,157 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.dataartisans.flink.dataflow.examples.streaming; + +import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSocketSource; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.Read; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple; +import com.google.cloud.dataflow.sdk.transforms.windowing.*; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PBegin; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import org.joda.time.Duration; + +/** + * To run the example, first open two sockets on two terminals by executing the commands: + *

  • + *
  • + * nc -lk 9999, and + *
  • + *
  • + * nc -lk 9998 + *
  • + * + * and then launch the example. Now whatever you type in the terminal is going to be + * the input to the program. + * */ +public class JoinExamples { + + static PCollection joinEvents(PCollection streamA, + PCollection streamB) throws Exception { + + final TupleTag firstInfoTag = new TupleTag(); + final TupleTag secondInfoTag = new TupleTag(); + + // transform both input collections to tuple collections, where the keys are country + // codes in both cases. + PCollection> firstInfo = streamA.apply( + ParDo.of(new ExtractEventDataFn())); + PCollection> secondInfo = streamB.apply( + ParDo.of(new ExtractEventDataFn())); + + // country code 'key' -> CGBKR (, ) + PCollection> kvpCollection = KeyedPCollectionTuple + .of(firstInfoTag, firstInfo) + .and(secondInfoTag, secondInfo) + .apply(CoGroupByKey.create()); + + // Process the CoGbkResult elements generated by the CoGroupByKey transform. + // country code 'key' -> string of , + PCollection> finalResultCollection = + kvpCollection.apply(ParDo.named("Process").of( + new DoFn, KV>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + String key = e.getKey(); + + String defaultA = "NO_VALUE"; + + // the following getOnly is a bit tricky because it expects to have + // EXACTLY ONE value in the corresponding stream and for the corresponding key. + + String lineA = e.getValue().getOnly(firstInfoTag, defaultA); + for (String lineB : c.element().getValue().getAll(secondInfoTag)) { + // Generate a string that combines information from both collection values + c.output(KV.of(key, "Value A: " + lineA + " - Value B: " + lineB)); + } + } + })); + + return finalResultCollection + .apply(ParDo.named("Format").of(new DoFn, String>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + String result = c.element().getKey() + " -> " + c.element().getValue(); + System.out.println(result); + c.output(result); + } + })); + } + + static class ExtractEventDataFn extends DoFn> { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + String line = c.element().toLowerCase(); + String key = line.split("\\s")[0]; + c.output(KV.of(key, line)); + } + } + + private static interface Options extends WindowedWordCount.StreamingWordCountOptions { + + } + + public static void main(String[] args) throws Exception { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + // make it a streaming example. + options.setStreaming(true); + options.setRunner(FlinkPipelineRunner.class); + + PTransform> readSourceA = + Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("FirstStream"); + PTransform> readSourceB = + Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)).named("SecondStream"); + + WindowFn windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize())); + + Pipeline p = Pipeline.create(options); + + // the following two 'applys' create multiple inputs to our pipeline, one for each + // of our two input sources. + PCollection streamA = p.apply(readSourceA) + .apply(Window.into(windowFn) + .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()); + PCollection streamB = p.apply(readSourceB) + .apply(Window.into(windowFn) + .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()); + + PCollection formattedResults = joinEvents(streamA, streamB); + formattedResults.apply(TextIO.Write.to("./outputJoin.txt")); + p.run(); + } + +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java new file mode 100644 index 000000000000..46c9bd602462 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java @@ -0,0 +1,138 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.examples.streaming; + +import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedFlinkSource; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.Read; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.io.UnboundedSource; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.*; +import com.google.cloud.dataflow.sdk.transforms.windowing.*; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer082; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.joda.time.Duration; + +import java.util.Properties; + +public class KafkaWindowedWordCountExample { + + static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from + static final String KAFKA_BROKER = "localhost:9092"; // Default kafka broker to contact + static final String GROUP_ID = "myGroup"; // Default groupId + static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka + + public static class ExtractWordsFn extends DoFn { + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + if (c.element().trim().isEmpty()) { + emptyLines.addValue(1L); + } + + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + public static class FormatAsStringFn extends DoFn, String> { + @Override + public void processElement(ProcessContext c) { + String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); + System.out.println(row); + c.output(row); + } + } + + public static interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions { + @Description("The Kafka topic to read from") + @Default.String(KAFKA_TOPIC) + String getKafkaTopic(); + + void setKafkaTopic(String value); + + @Description("The Kafka Broker to read from") + @Default.String(KAFKA_BROKER) + String getBroker(); + + void setBroker(String value); + + @Description("The Zookeeper server to connect to") + @Default.String(ZOOKEEPER) + String getZookeeper(); + + void setZookeeper(String value); + + @Description("The groupId") + @Default.String(GROUP_ID) + String getGroup(); + + void setGroup(String value); + + } + + public static void main(String[] args) { + PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class); + KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class); + options.setJobName("KafkaExample"); + options.setStreaming(true); + options.setRunner(FlinkPipelineRunner.class); + + System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() ); + Pipeline pipeline = Pipeline.create(options); + + Properties p = new Properties(); + p.setProperty("zookeeper.connect", options.getZookeeper()); + p.setProperty("bootstrap.servers", options.getBroker()); + p.setProperty("group.id", options.getGroup()); + + // this is the Flink consumer that reads the input to + // the program from a kafka topic. + FlinkKafkaConsumer082 kafkaConsumer = new FlinkKafkaConsumer082<>( + options.getKafkaTopic(), + new SimpleStringSchema(), p); + + PCollection words = pipeline + .apply(Read.from(new UnboundedFlinkSource(options, kafkaConsumer)).named("StreamingWordCount")) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Window.into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize()))) + .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()); + + PCollection> wordCounts = + words.apply(Count.perElement()); + + wordCounts.apply(ParDo.of(new FormatAsStringFn())) + .apply(TextIO.Write.to("./outputKafka.txt")); + + pipeline.run(); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java new file mode 100644 index 000000000000..1d4a44b494fd --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java @@ -0,0 +1,126 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.examples.streaming; + +import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSocketSource; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.*; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.*; +import com.google.cloud.dataflow.sdk.transforms.windowing.*; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * To run the example, first open a socket on a terminal by executing the command: + *
  • + *
  • + * nc -lk 9999 + *
  • + * + * and then launch the example. Now whatever you type in the terminal is going to be + * the input to the program. + * */ +public class WindowedWordCount { + + private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class); + + static final long WINDOW_SIZE = 10; // Default window duration in seconds + static final long SLIDE_SIZE = 5; // Default window slide in seconds + + static class FormatAsStringFn extends DoFn, String> { + @Override + public void processElement(ProcessContext c) { + String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); + c.output(row); + } + } + + static class ExtractWordsFn extends DoFn { + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + if (c.element().trim().isEmpty()) { + emptyLines.addValue(1L); + } + + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + public static interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options { + @Description("Sliding window duration, in seconds") + @Default.Long(WINDOW_SIZE) + Long getWindowSize(); + + void setWindowSize(Long value); + + @Description("Window slide, in seconds") + @Default.Long(SLIDE_SIZE) + Long getSlide(); + + void setSlide(Long value); + } + + public static void main(String[] args) throws IOException { + StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class); + options.setStreaming(true); + options.setWindowSize(10L); + options.setSlide(5L); + options.setRunner(FlinkPipelineRunner.class); + + LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() + + " sec. and a slide of " + options.getSlide()); + + Pipeline pipeline = Pipeline.create(options); + + PCollection words = pipeline + .apply(Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("StreamingWordCount")) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Window.into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize())) + .every(Duration.standardSeconds(options.getSlide()))) + .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()); + + PCollection> wordCounts = + words.apply(Count.perElement()); + + wordCounts.apply(ParDo.of(new FormatAsStringFn())) + .apply(TextIO.Write.to("./outputWordCount.txt")); + + pipeline.run(); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java new file mode 100644 index 000000000000..8c0183ea6a1a --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java @@ -0,0 +1,152 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.values.PValue; +import org.apache.flink.api.java.ExecutionEnvironment; + +/** + * FlinkBatchPipelineTranslator knows how to translate Pipeline objects into Flink Jobs. + * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator} + */ +public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator { + + /** + * The necessary context in the case of a batch job. + */ + private final FlinkBatchTranslationContext batchContext; + + private int depth = 0; + + /** + * Composite transform that we want to translate before proceeding with other transforms. + */ + private PTransform currentCompositeTransform; + + public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) { + this.batchContext = new FlinkBatchTranslationContext(env, options); + } + + // -------------------------------------------------------------------------------------------- + // Pipeline Visitor Methods + // -------------------------------------------------------------------------------------------- + + @Override + public void enterCompositeTransform(TransformTreeNode node) { + System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); + + PTransform transform = node.getTransform(); + if (transform != null && currentCompositeTransform == null) { + + BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + if (translator != null) { + currentCompositeTransform = transform; + if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) { + // we can only optimize CoGroupByKey for input size 2 + currentCompositeTransform = null; + } + } + } + this.depth++; + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + if (transform != null && currentCompositeTransform == transform) { + + BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + if (translator != null) { + System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); + applyBatchTransform(transform, node, translator); + currentCompositeTransform = null; + } else { + throw new IllegalStateException("Attempted to translate composite transform " + + "but no translator was found: " + currentCompositeTransform); + } + } + this.depth--; + System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node)); + } + + @Override + public void visitTransform(TransformTreeNode node) { + System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node)); + if (currentCompositeTransform != null) { + // ignore it + return; + } + + // get the transformation corresponding to hte node we are + // currently visiting and translate it into its Flink alternative. + + PTransform transform = node.getTransform(); + BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + if (translator == null) { + System.out.println(node.getTransform().getClass()); + throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); + } + applyBatchTransform(transform, node, translator); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + // do nothing here + } + + private > void applyBatchTransform(PTransform transform, TransformTreeNode node, BatchTransformTranslator translator) { + if (this.batchContext == null) { + throw new IllegalStateException("The FlinkPipelineTranslator is not yet initialized."); + } + + @SuppressWarnings("unchecked") + T typedTransform = (T) transform; + + @SuppressWarnings("unchecked") + BatchTransformTranslator typedTranslator = (BatchTransformTranslator) translator; + + // create the applied PTransform on the batchContext + batchContext.setCurrentTransform(AppliedPTransform.of( + node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); + typedTranslator.translateNode(typedTransform, batchContext); + } + + /** + * A translator of a {@link PTransform}. + */ + public interface BatchTransformTranslator { + void translateNode(Type transform, FlinkBatchTranslationContext context); + } + + private static String genSpaces(int n) { + String s = ""; + for (int i = 0; i < n; i++) { + s += "| "; + } + return s; + } + + private static String formatNodeName(TransformTreeNode node) { + return node.toString().split("@")[1] + node.getTransform(); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java similarity index 82% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java rename to runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java index c1d78c02eedf..8f647300d2da 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java @@ -91,59 +91,59 @@ * Dataflow {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s to * Flink {@link org.apache.flink.api.java.DataSet}s */ -public class FlinkTransformTranslators { +public class FlinkBatchTransformTranslators { // -------------------------------------------------------------------------------------------- // Transform Translator Registry // -------------------------------------------------------------------------------------------- @SuppressWarnings("rawtypes") - private static final Map, FlinkPipelineTranslator.TransformTranslator> TRANSLATORS = new HashMap<>(); + private static final Map, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>(); // register the known translators static { - TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslator()); + TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch()); - TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator()); + TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch()); // we don't need this because we translate the Combine.PerKey directly //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator()); - TRANSLATORS.put(Create.Values.class, new CreateTranslator()); + TRANSLATORS.put(Create.Values.class, new CreateTranslatorBatch()); - TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); + TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch()); - TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslator()); + TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslatorBatch()); // TODO we're currently ignoring windows here but that has to change in the future - TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator()); + TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); - TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator()); - TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslator()); + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); + TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch()); - TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslator()); + TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslatorBatch()); - TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslator()); - TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslator()); + TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslatorBatch()); + TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslatorBatch()); - TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslator()); - TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslator()); + TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch()); + TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslatorBatch()); - TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslator()); - TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslator()); + TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslatorBatch()); + TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslatorBatch()); // Flink-specific - TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslator()); + TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslatorBatch()); } - public static FlinkPipelineTranslator.TransformTranslator getTranslator(PTransform transform) { + public static FlinkBatchPipelineTranslator.BatchTransformTranslator getTranslator(PTransform transform) { return TRANSLATORS.get(transform.getClass()); } - private static class ReadSourceTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class ReadSourceTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(Read.Bounded transform, TranslationContext context) { + public void translateNode(Read.Bounded transform, FlinkBatchTranslationContext context) { String name = transform.getName(); BoundedSource source = transform.getSource(); PCollection output = context.getOutput(transform); @@ -157,11 +157,11 @@ public void translateNode(Read.Bounded transform, TranslationContext context) } } - private static class AvroIOReadTranslator implements FlinkPipelineTranslator.TransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslator.class); + private static class AvroIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslatorBatch.class); @Override - public void translateNode(AvroIO.Read.Bound transform, TranslationContext context) { + public void translateNode(AvroIO.Read.Bound transform, FlinkBatchTranslationContext context) { String path = transform.getFilepattern(); String name = transform.getName(); // Schema schema = transform.getSchema(); @@ -190,11 +190,11 @@ public void translateNode(AvroIO.Read.Bound transform, TranslationContext con } } - private static class AvroIOWriteTranslator implements FlinkPipelineTranslator.TransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslator.class); + private static class AvroIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslatorBatch.class); @Override - public void translateNode(AvroIO.Write.Bound transform, TranslationContext context) { + public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslationContext context) { DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); String filenamePrefix = transform.getFilenamePrefix(); String filenameSuffix = transform.getFilenameSuffix(); @@ -228,11 +228,11 @@ public void translateNode(AvroIO.Write.Bound transform, TranslationContext co } } - private static class TextIOReadTranslator implements FlinkPipelineTranslator.TransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslator.class); + private static class TextIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslatorBatch.class); @Override - public void translateNode(TextIO.Read.Bound transform, TranslationContext context) { + public void translateNode(TextIO.Read.Bound transform, FlinkBatchTranslationContext context) { String path = transform.getFilepattern(); String name = transform.getName(); @@ -246,18 +246,17 @@ public void translateNode(TextIO.Read.Bound transform, TranslationContex PValue output = context.getOutput(transform); TypeInformation typeInformation = context.getTypeInfo(output); - DataSource source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name); context.setOutputDataSet(output, source); } } - private static class TextIOWriteTranslator implements FlinkPipelineTranslator.TransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslator.class); + private static class TextIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class); @Override - public void translateNode(TextIO.Write.Bound transform, TranslationContext context) { + public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslationContext context) { PValue input = context.getInput(transform); DataSet inputDataSet = context.getInputDataSet(input); @@ -281,19 +280,19 @@ public void translateNode(TextIO.Write.Bound transform, TranslationContext co } } - private static class ConsoleIOWriteTranslator implements FlinkPipelineTranslator.TransformTranslator { + private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator { @Override - public void translateNode(ConsoleIO.Write.Bound transform, TranslationContext context) { - PValue input = context.getInput(transform); + public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) { + PValue input = (PValue) context.getInput(transform); DataSet inputDataSet = context.getInputDataSet(input); inputDataSet.printOnTaskManager(transform.getName()); } } - private static class WriteSinkTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class WriteSinkTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(Write.Bound transform, TranslationContext context) { + public void translateNode(Write.Bound transform, FlinkBatchTranslationContext context) { String name = transform.getName(); PValue input = context.getInput(transform); DataSet inputDataSet = context.getInputDataSet(input); @@ -302,10 +301,10 @@ public void translateNode(Write.Bound transform, TranslationContext context) } } - private static class GroupByKeyOnlyTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class GroupByKeyOnlyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(GroupByKey.GroupByKeyOnly transform, TranslationContext context) { + public void translateNode(GroupByKey.GroupByKeyOnly transform, FlinkBatchTranslationContext context) { DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); @@ -320,12 +319,12 @@ public void translateNode(GroupByKey.GroupByKeyOnly transform, Translation } /** - * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslator} + * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslatorBatch} */ - private static class GroupByKeyTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class GroupByKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(GroupByKey transform, TranslationContext context) { + public void translateNode(GroupByKey transform, FlinkBatchTranslationContext context) { DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); @@ -340,10 +339,10 @@ public void translateNode(GroupByKey transform, TranslationContext context } } - private static class CombinePerKeyTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class CombinePerKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(Combine.PerKey transform, TranslationContext context) { + public void translateNode(Combine.PerKey transform, FlinkBatchTranslationContext context) { DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); @SuppressWarnings("unchecked") @@ -407,11 +406,11 @@ public void translateNode(Combine.PerKey transform, TranslationContex // } // } - private static class ParDoBoundTranslator implements FlinkPipelineTranslator.TransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslator.class); + private static class ParDoBoundTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorBatch.class); @Override - public void translateNode(ParDo.Bound transform, TranslationContext context) { + public void translateNode(ParDo.Bound transform, FlinkBatchTranslationContext context) { DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); final DoFn doFn = transform.getFn(); @@ -427,11 +426,11 @@ public void translateNode(ParDo.Bound transform, TranslationContext con } } - private static class ParDoBoundMultiTranslator implements FlinkPipelineTranslator.TransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslator.class); + private static class ParDoBoundMultiTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslatorBatch.class); @Override - public void translateNode(ParDo.BoundMulti transform, TranslationContext context) { + public void translateNode(ParDo.BoundMulti transform, FlinkBatchTranslationContext context) { DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); final DoFn doFn = transform.getFn(); @@ -478,10 +477,10 @@ public void translateNode(ParDo.BoundMulti transform, TranslationContex } } - private static class FlattenPCollectionTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class FlattenPCollectionTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(Flatten.FlattenPCollectionList transform, TranslationContext context) { + public void translateNode(Flatten.FlattenPCollectionList transform, FlinkBatchTranslationContext context) { List> allInputs = context.getInput(transform).getAll(); DataSet result = null; for(PCollection collection : allInputs) { @@ -496,19 +495,19 @@ public void translateNode(Flatten.FlattenPCollectionList transform, Translati } } - private static class CreatePCollectionViewTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class CreatePCollectionViewTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(View.CreatePCollectionView transform, TranslationContext context) { + public void translateNode(View.CreatePCollectionView transform, FlinkBatchTranslationContext context) { DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); PCollectionView input = transform.apply(null); context.setSideInputDataSet(input, inputDataSet); } } - private static class CreateTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class CreateTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(Create.Values transform, TranslationContext context) { + public void translateNode(Create.Values transform, FlinkBatchTranslationContext context) { TypeInformation typeInformation = context.getOutputTypeInfo(); Iterable elements = transform.getElements(); @@ -538,7 +537,7 @@ public void translateNode(Create.Values transform, TranslationContext conte private static void transformSideInputs(List> sideInputs, MapPartitionOperator outputDataSet, - TranslationContext context) { + FlinkBatchTranslationContext context) { // get corresponding Flink broadcast DataSets for(PCollectionView input : sideInputs) { DataSet broadcastSet = context.getSideInputDataSet(input); @@ -551,10 +550,10 @@ private static void transformSideInputs(List> sideInputs, * Special composite transform translator. Only called if the CoGroup is two dimensional. * @param */ - private static class CoGroupByKeyTranslator implements FlinkPipelineTranslator.TransformTranslator> { + private static class CoGroupByKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(CoGroupByKey transform, TranslationContext context) { + public void translateNode(CoGroupByKey transform, FlinkBatchTranslationContext context) { KeyedPCollectionTuple input = context.getInput(transform); CoGbkResultSchema schema = input.getCoGbkResultSchema(); @@ -590,5 +589,5 @@ public void translateNode(CoGroupByKey transform, TranslationContext context) // Miscellaneous // -------------------------------------------------------------------------------------------- - private FlinkTransformTranslators() {} + private FlinkBatchTransformTranslators() {} } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java similarity index 96% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java rename to runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java index af461094fa3f..1072fa32c551 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java @@ -37,7 +37,7 @@ import java.util.HashMap; import java.util.Map; -public class TranslationContext { +public class FlinkBatchTranslationContext { private final Map> dataSets; private final Map, DataSet> broadcastDataSets; @@ -49,7 +49,7 @@ public class TranslationContext { // ------------------------------------------------------------------------ - public TranslationContext(ExecutionEnvironment env, PipelineOptions options) { + public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) { this.env = env; this.options = options; this.dataSets = new HashMap<>(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java index 92b9135cccea..e5c85451b2b6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java @@ -7,8 +7,6 @@ * * 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. @@ -18,151 +16,10 @@ package com.dataartisans.flink.dataflow.translation; import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor; -import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; -import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; -import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; -import com.google.cloud.dataflow.sdk.values.PValue; -import org.apache.flink.api.java.ExecutionEnvironment; - -/** - * FlinkPipelineTranslator knows how to translate Pipeline objects into Flink Jobs. - * - * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator} - */ -public class FlinkPipelineTranslator implements PipelineVisitor { - - private final TranslationContext context; - - private int depth = 0; - - /** - * Composite transform that we want to translate before proceeding with other transforms - */ - private PTransform currentCompositeTransform; - - public FlinkPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) { - this.context = new TranslationContext(env, options); - } +public abstract class FlinkPipelineTranslator implements Pipeline.PipelineVisitor { public void translate(Pipeline pipeline) { pipeline.traverseTopologically(this); } - - - // -------------------------------------------------------------------------------------------- - // Pipeline Visitor Methods - // -------------------------------------------------------------------------------------------- - - private static String genSpaces(int n) { - String s = ""; - for(int i = 0; i < n; i++) { - s += "| "; - } - return s; - } - - private static String formatNodeName(TransformTreeNode node) { - return node.toString().split("@")[1] + node.getTransform(); - } - - @Override - public void enterCompositeTransform(TransformTreeNode node) { - System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); - PTransform transform = node.getTransform(); - - if (transform != null && currentCompositeTransform == null) { - TransformTranslator translator = FlinkTransformTranslators.getTranslator(transform); - - if (translator != null) { - currentCompositeTransform = transform; - - if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) { - // we can only optimize CoGroupByKey for input size 2 - currentCompositeTransform = null; - } - } - } - - this.depth++; - } - - @Override - public void leaveCompositeTransform(TransformTreeNode node) { - PTransform transform = node.getTransform(); - - if (transform != null) { - TransformTranslator translator = FlinkTransformTranslators.getTranslator(transform); - - if (currentCompositeTransform == transform) { - if (translator != null) { - System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); - applyTransform(transform, node, translator); - currentCompositeTransform = null; - } else { - throw new IllegalStateException("Attempted to translate composite transform " + - "but no translator was found: " + currentCompositeTransform); - } - } - } - - this.depth--; - System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node)); - } - - @Override - public void visitTransform(TransformTreeNode node) { - System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node)); - if (currentCompositeTransform != null) { - // ignore it - return; - } - - // the transformation applied in this node - PTransform transform = node.getTransform(); - - // the translator to the Flink operation(s) - TransformTranslator translator = FlinkTransformTranslators.getTranslator(transform); - - if (translator == null) { - System.out.println(node.getTransform().getClass()); - throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); - } - - applyTransform(transform, node, translator); - } - - @Override - public void visitValue(PValue value, TransformTreeNode producer) { - // do nothing here - } - - /** - * Utility method to define a generic variable to cast the translator and the transform to. - */ - private > void applyTransform(PTransform transform, TransformTreeNode node, TransformTranslator translator) { - - @SuppressWarnings("unchecked") - T typedTransform = (T) transform; - - @SuppressWarnings("unchecked") - TransformTranslator typedTranslator = (TransformTranslator) translator; - - // create the applied PTransform on the context - context.setCurrentTransform(AppliedPTransform.of( - node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); - - typedTranslator.translateNode(typedTransform, context); - } - - /** - * A translator of a {@link PTransform}. - */ - public interface TransformTranslator { - - void translateNode(Type transform, TranslationContext context); - } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java new file mode 100644 index 000000000000..c8760c7ecbe1 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java @@ -0,0 +1,138 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; +import com.google.cloud.dataflow.sdk.values.PValue; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator { + + /** The necessary context in the case of a straming job. */ + private final FlinkStreamingTranslationContext streamingContext; + + private int depth = 0; + + /** Composite transform that we want to translate before proceeding with other transforms. */ + private PTransform currentCompositeTransform; + + public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, PipelineOptions options) { + this.streamingContext = new FlinkStreamingTranslationContext(env, options); + } + + // -------------------------------------------------------------------------------------------- + // Pipeline Visitor Methods + // -------------------------------------------------------------------------------------------- + + @Override + public void enterCompositeTransform(TransformTreeNode node) { + System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); + + PTransform transform = node.getTransform(); + if (transform != null && currentCompositeTransform == null) { + + StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); + if (translator != null) { + currentCompositeTransform = transform; + } + } + this.depth++; + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + if (transform != null && currentCompositeTransform == transform) { + + StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); + if (translator != null) { + System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); + applyStreamingTransform(transform, node, translator); + currentCompositeTransform = null; + } else { + throw new IllegalStateException("Attempted to translate composite transform " + + "but no translator was found: " + currentCompositeTransform); + } + } + this.depth--; + System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node)); + } + + @Override + public void visitTransform(TransformTreeNode node) { + System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node)); + if (currentCompositeTransform != null) { + // ignore it + return; + } + + // get the transformation corresponding to hte node we are + // currently visiting and translate it into its Flink alternative. + + PTransform transform = node.getTransform(); + StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); + if (translator == null) { + System.out.println(node.getTransform().getClass()); + throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); + } + applyStreamingTransform(transform, node, translator); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + // do nothing here + } + + private > void applyStreamingTransform(PTransform transform, TransformTreeNode node, StreamTransformTranslator translator) { + if (this.streamingContext == null) { + throw new IllegalStateException("The FlinkPipelineTranslator is not yet initialized."); + } + + @SuppressWarnings("unchecked") + T typedTransform = (T) transform; + + @SuppressWarnings("unchecked") + StreamTransformTranslator typedTranslator = (StreamTransformTranslator) translator; + + // create the applied PTransform on the batchContext + streamingContext.setCurrentTransform(AppliedPTransform.of( + node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); + typedTranslator.translateNode(typedTransform, streamingContext); + } + + /** + * A translator of a {@link PTransform}. + */ + public interface StreamTransformTranslator { + void translateNode(Type transform, FlinkStreamingTranslationContext context); + } + + private static String genSpaces(int n) { + String s = ""; + for (int i = 0; i < n; i++) { + s += "| "; + } + return s; + } + + private static String formatNodeName(TransformTreeNode node) { + return node.toString().split("@")[1] + node.getTransform(); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java new file mode 100644 index 000000000000..4c8cd4bf496f --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java @@ -0,0 +1,356 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation; + +import com.dataartisans.flink.dataflow.translation.functions.UnionCoder; +import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.*; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedFlinkSource; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSourceWrapper; +import com.google.api.client.util.Maps; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.io.Read; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.transforms.*; +import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; +import com.google.cloud.dataflow.sdk.transforms.windowing.*; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowingStrategy; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.collect.Lists; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.streaming.api.datastream.*; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; + +/** + *

    + * Coder entryCoder = pCollection.getCoder(); + * if (!(entryCoder instanceof KvCoder)) { + * throw new IllegalArgumentException("PCollection does not use a KvCoder"); + * } + */ +public class FlinkStreamingTransformTranslators { + + // -------------------------------------------------------------------------------------------- + // Transform Translator Registry + // -------------------------------------------------------------------------------------------- + + @SuppressWarnings("rawtypes") + private static final Map, FlinkStreamingPipelineTranslator.StreamTransformTranslator> TRANSLATORS = new HashMap<>(); + + // here you can find all the available translators. + static { + TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator()); + TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator()); + TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator()); + TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslator()); + TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator()); + TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator()); + TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator()); + + } + + public static FlinkStreamingPipelineTranslator.StreamTransformTranslator getTranslator(PTransform transform) { + FlinkStreamingPipelineTranslator.StreamTransformTranslator translator = TRANSLATORS.get(transform.getClass()); + return translator; + } + + // -------------------------------------------------------------------------------------------- + // Transformation Implementations + // -------------------------------------------------------------------------------------------- + + private static class TextIOWriteBoundStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class); + + @Override + public void translateNode(TextIO.Write.Bound transform, FlinkStreamingTranslationContext context) { + PValue input = context.getInput(transform); + DataStream> inputDataStream = context.getInputDataStream(input); + + String filenamePrefix = transform.getFilenamePrefix(); + String filenameSuffix = transform.getFilenameSuffix(); + boolean needsValidation = transform.needsValidation(); + int numShards = transform.getNumShards(); + String shardNameTemplate = transform.getShardNameTemplate(); + + // TODO: Implement these. We need Flink support for this. + LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation); + LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix); + LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); + + DataStream dataSink = inputDataStream.flatMap(new FlatMapFunction, String>() { + @Override + public void flatMap(WindowedValue value, Collector out) throws Exception { + out.collect(value.getValue().toString()); + } + }); + DataStreamSink output = dataSink.writeAsText(filenamePrefix, FileSystem.WriteMode.OVERWRITE); + + if (numShards > 0) { + output.setParallelism(numShards); + } + } + } + + private static class UnboundedReadSourceTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(Read.Unbounded transform, FlinkStreamingTranslationContext context) { + PCollection output = context.getOutput(transform); + + DataStream> source = null; + if (transform.getSource().getClass().equals(UnboundedFlinkSource.class)) { + UnboundedFlinkSource flinkSource = (UnboundedFlinkSource) transform.getSource(); + source = context.getExecutionEnvironment() + .addSource(flinkSource.getFlinkSource()) + .flatMap(new FlatMapFunction>() { + @Override + public void flatMap(String s, Collector> collector) throws Exception { + collector.collect(WindowedValue.of(s, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + } + }); + } else { + source = context.getExecutionEnvironment() + .addSource(new UnboundedSourceWrapper<>(context.getPipelineOptions(), transform)); + } + context.setOutputDataStream(output, source); + } + } + + private static class ParDoBoundStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(ParDo.Bound transform, FlinkStreamingTranslationContext context) { + PCollection output = context.getOutput(transform); + + final WindowingStrategy windowingStrategy = + (WindowingStrategy) + context.getOutput(transform).getWindowingStrategy(); + + WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(), windowingStrategy.getWindowFn().windowCoder()); + CoderTypeInformation> outputWindowedValueCoder = new CoderTypeInformation<>(outputStreamCoder); + + FlinkParDoBoundWrapper doFnWrapper = new FlinkParDoBoundWrapper<>(context.getPipelineOptions(), windowingStrategy, transform.getFn()); + DataStream> inputDataStream = context.getInputDataStream(context.getInput(transform)); + SingleOutputStreamOperator, ?> outDataStream = inputDataStream.flatMap(doFnWrapper).returns(outputWindowedValueCoder); + + context.setOutputDataStream(context.getOutput(transform), outDataStream); + } + } + + public static class WindowBoundTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(Window.Bound transform, FlinkStreamingTranslationContext context) { + PValue input = context.getInput(transform); + DataStream> inputDataStream = context.getInputDataStream(input); + + final WindowingStrategy windowingStrategy = + (WindowingStrategy) + context.getOutput(transform).getWindowingStrategy(); + + final WindowFn windowFn = windowingStrategy.getWindowFn(); + + WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder( + context.getInput(transform).getCoder(), windowingStrategy.getWindowFn().windowCoder()); + CoderTypeInformation> outputWindowedValueCoder = + new CoderTypeInformation<>(outputStreamCoder); + + final FlinkParDoBoundWrapper windowDoFnAssigner = new FlinkParDoBoundWrapper<>( + context.getPipelineOptions(), windowingStrategy, createWindowAssigner(windowFn)); + + SingleOutputStreamOperator, ?> windowedStream = + inputDataStream.flatMap(windowDoFnAssigner).returns(outputWindowedValueCoder); + context.setOutputDataStream(context.getOutput(transform), windowedStream); + } + + private static DoFn createWindowAssigner(final WindowFn windowFn) { + return new DoFn() { + + @Override + public void processElement(final ProcessContext c) throws Exception { + Collection windows = windowFn.assignWindows( + windowFn.new AssignContext() { + @Override + public T element() { + return c.element(); + } + + @Override + public Instant timestamp() { + return c.timestamp(); + } + + @Override + public Collection windows() { + return c.windowingInternals().windows(); + } + }); + + c.windowingInternals().outputWindowedValue( + c.element(), c.timestamp(), windows, c.pane()); + } + }; + } + } + + public static class GroupByKeyTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(GroupByKey transform, FlinkStreamingTranslationContext context) { + PValue input = context.getInput(transform); + + DataStream>> inputDataStream = context.getInputDataStream(input); + KvCoder inputKvCoder = (KvCoder) context.getInput(transform).getCoder(); + + KeyedStream>, K> groupByKStream = FlinkGroupByKeyWrapper + .groupStreamByKey(inputDataStream, inputKvCoder); + + DataStream>>> groupedByKNWstream = + FlinkGroupAlsoByWindowWrapper.createForIterable(context.getPipelineOptions(), + context.getInput(transform), groupByKStream); + + context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream); + } + } + + public static class CombinePerKeyTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(Combine.PerKey transform, FlinkStreamingTranslationContext context) { + PValue input = context.getInput(transform); + + DataStream>> inputDataStream = context.getInputDataStream(input); + KvCoder inputKvCoder = (KvCoder) context.getInput(transform).getCoder(); + KvCoder outputKvCoder = (KvCoder) context.getOutput(transform).getCoder(); + + KeyedStream>, K> groupByKStream = FlinkGroupByKeyWrapper + .groupStreamByKey(inputDataStream, inputKvCoder); + + Combine.KeyedCombineFn combineFn = (Combine.KeyedCombineFn) transform.getFn(); + DataStream>> groupedByKNWstream = + FlinkGroupAlsoByWindowWrapper.create(context.getPipelineOptions(), + context.getInput(transform), groupByKStream, combineFn, outputKvCoder); + + context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream); + } + } + + public static class FlattenPCollectionTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(Flatten.FlattenPCollectionList transform, FlinkStreamingTranslationContext context) { + List> allInputs = context.getInput(transform).getAll(); + DataStream result = null; + for (PCollection collection : allInputs) { + DataStream current = context.getInputDataStream(collection); + result = (result == null) ? current : result.union(current); + } + context.setOutputDataStream(context.getOutput(transform), result); + } + } + + public static class ParDoBoundMultiStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + private final int MAIN_TAG_INDEX = 0; + + @Override + public void translateNode(ParDo.BoundMulti transform, FlinkStreamingTranslationContext context) { + + // we assume that the transformation does not change the windowing strategy. + WindowingStrategy windowingStrategy = context.getInput(transform).getWindowingStrategy(); + + Map, PCollection> outputs = context.getOutput(transform).getAll(); + Map, Integer> tagsToLabels = transformTupleTagsToLabels( + transform.getMainOutputTag(), outputs.keySet()); + + UnionCoder intermUnionCoder = getIntermUnionCoder(outputs.values()); + WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder( + intermUnionCoder, windowingStrategy.getWindowFn().windowCoder()); + + CoderTypeInformation> intermWindowedValueCoder = + new CoderTypeInformation<>(outputStreamCoder); + + FlinkParDoBoundMultiWrapper doFnWrapper = new FlinkParDoBoundMultiWrapper<>( + context.getPipelineOptions(), windowingStrategy, transform.getFn(), + transform.getMainOutputTag(), tagsToLabels); + + DataStream> inputDataStream = context.getInputDataStream(context.getInput(transform)); + SingleOutputStreamOperator, ?> intermDataStream = + inputDataStream.flatMap(doFnWrapper).returns(intermWindowedValueCoder); + + for (Map.Entry, PCollection> output : outputs.entrySet()) { + final int outputTag = tagsToLabels.get(output.getKey()); + + WindowedValue.WindowedValueCoder coderForTag = WindowedValue.getFullCoder( + output.getValue().getCoder(), + windowingStrategy.getWindowFn().windowCoder()); + + CoderTypeInformation> windowedValueCoder = + new CoderTypeInformation(coderForTag); + + context.setOutputDataStream(output.getValue(), + intermDataStream.filter(new FilterFunction>() { + @Override + public boolean filter(WindowedValue value) throws Exception { + return value.getValue().getUnionTag() == outputTag; + } + }).flatMap(new FlatMapFunction, WindowedValue>() { + @Override + public void flatMap(WindowedValue value, Collector> collector) throws Exception { + collector.collect(WindowedValue.of( + value.getValue().getValue(), + value.getTimestamp(), + value.getWindows(), + value.getPane())); + } + }).returns(windowedValueCoder)); + } + } + + private Map, Integer> transformTupleTagsToLabels(TupleTag mainTag, Set> secondaryTags) { + Map, Integer> tagToLabelMap = Maps.newHashMap(); + tagToLabelMap.put(mainTag, MAIN_TAG_INDEX); + int count = MAIN_TAG_INDEX + 1; + for (TupleTag tag : secondaryTags) { + if (!tagToLabelMap.containsKey(tag)) { + tagToLabelMap.put(tag, count++); + } + } + return tagToLabelMap; + } + + private UnionCoder getIntermUnionCoder(Collection> taggedCollections) { + List> outputCoders = Lists.newArrayList(); + for (PCollection coll : taggedCollections) { + outputCoders.add(coll.getCoder()); + } + return UnionCoder.of(outputCoders); + } + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java new file mode 100644 index 000000000000..83ea5753abde --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java @@ -0,0 +1,86 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.values.*; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import java.util.HashMap; +import java.util.Map; + +public class FlinkStreamingTranslationContext { + + private final StreamExecutionEnvironment env; + private final PipelineOptions options; + + /** + * Keeps a mapping between the output value of the PTransform (in Dataflow) and the + * Flink Operator that produced it, after the translation of the correspondinf PTransform + * to its Flink equivalent. + * */ + private final Map> dataStreams; + + private AppliedPTransform currentTransform; + + public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) { + this.env = env; + this.options = options; + this.dataStreams = new HashMap<>(); + } + + public StreamExecutionEnvironment getExecutionEnvironment() { + return env; + } + + public PipelineOptions getPipelineOptions() { + return options; + } + + @SuppressWarnings("unchecked") + public DataStream getInputDataStream(PValue value) { + return (DataStream) dataStreams.get(value); + } + + public void setOutputDataStream(PValue value, DataStream set) { + if (!dataStreams.containsKey(value)) { + dataStreams.put(value, set); + } + } + + /** + * Sets the AppliedPTransform which carries input/output. + * @param currentTransform + */ + public void setCurrentTransform(AppliedPTransform currentTransform) { + this.currentTransform = currentTransform; + } + + @SuppressWarnings("unchecked") + public I getInput(PTransform transform) { + I input = (I) currentTransform.getInput(); + return input; + } + + @SuppressWarnings("unchecked") + public O getOutput(PTransform transform) { + O output = (O) currentTransform.getOutput(); + return output; + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java similarity index 97% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java rename to runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java index ade826d82c83..e43358944564 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java @@ -30,7 +30,7 @@ * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for * {@link com.google.cloud.dataflow.sdk.coders.Coder}. */ -public class CoderComperator extends TypeComparator { +public class CoderComparator extends TypeComparator { private Coder coder; @@ -42,7 +42,7 @@ public class CoderComperator extends TypeComparator { // For storing the Reference in encoded form private transient InspectableByteArrayOutputStream referenceBuffer; - public CoderComperator(Coder coder) { + public CoderComparator(Coder coder) { this.coder = coder; buffer1 = new InspectableByteArrayOutputStream(); buffer2 = new InspectableByteArrayOutputStream(); @@ -51,11 +51,9 @@ public CoderComperator(Coder coder) { private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - buffer1 = new InspectableByteArrayOutputStream(); buffer2 = new InspectableByteArrayOutputStream(); referenceBuffer = new InspectableByteArrayOutputStream(); - } @Override @@ -97,7 +95,7 @@ public boolean equalToReference(T candidate) { @Override public int compareToReference(TypeComparator other) { - InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComperator) other).referenceBuffer; + InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComparator) other).referenceBuffer; byte[] arr = referenceBuffer.getBuffer(); byte[] arrOther = otherReferenceBuffer.getBuffer(); @@ -202,7 +200,7 @@ public boolean invertNormalizedKey() { @Override public TypeComparator duplicate() { - return new CoderComperator<>(coder); + return new CoderComparator<>(coder); } @Override diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java index 56192cd8e2aa..80e451a0b558 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java @@ -32,12 +32,12 @@ */ public class CoderTypeInformation extends TypeInformation implements AtomicType { - private Coder coder; + private final Coder coder; @SuppressWarnings("unchecked") public CoderTypeInformation(Coder coder) { - this.coder = coder; Preconditions.checkNotNull(coder); + this.coder = coder; } @Override @@ -112,6 +112,6 @@ public String toString() { @Override public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) { - return new CoderComperator<>(coder); + return new CoderComparator<>(coder); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java index 971547764ecf..f739397a4846 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java @@ -137,9 +137,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; CoderTypeSerializer that = (CoderTypeSerializer) o; - return coder.equals(that.coder); - } @Override diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index 940dba6b85f4..815569d09d85 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -259,6 +259,6 @@ public int extractKeys(Object record, Object[] target, int index) { @Override public TypeComparator[] getFlatComparators() { - return new TypeComparator[] {new CoderComperator<>(keyCoder)}; + return new TypeComparator[] {new CoderComparator<>(keyCoder)}; } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java index 2096e27d5d5e..7ce484ac10d2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java @@ -109,5 +109,4 @@ private VoidValue() {} public static VoidValue INSTANCE = new VoidValue(); } - } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java index 8c9c59ca54b1..afb15da51fa0 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java @@ -18,7 +18,6 @@ package com.dataartisans.flink.dataflow.translation.wrappers; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.api.client.util.Lists; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.BoundedSource; import com.google.cloud.dataflow.sdk.io.Source; @@ -34,6 +33,7 @@ import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; +import java.util.ArrayList; import java.util.List; /** @@ -116,7 +116,7 @@ public SourceInputSplit[] createInputSplits(int numSplits) throws IOException desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits; List> shards = initialSource.splitIntoBundles(desiredSizeBytes, options); - List> splits = Lists.newArrayList(); + List> splits = new ArrayList>(); int splitCount = 0; for (Source shard: shards) { splits.add(new SourceInputSplit<>(shard, splitCount++)); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java new file mode 100644 index 000000000000..53bb1778c227 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -0,0 +1,274 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming; + +import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; +import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; +import com.google.cloud.dataflow.sdk.util.*; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.base.Throwables; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.accumulators.AccumulatorHelper; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; +import org.joda.time.format.PeriodFormat; + +import java.util.Collection; + +public abstract class FlinkAbstractParDoWrapper extends RichFlatMapFunction, WindowedValue> { + + private final DoFn doFn; + private final WindowingStrategy windowingStrategy; + private transient PipelineOptions options; + + private DoFnProcessContext context; + + public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy windowingStrategy, DoFn doFn) { + Preconditions.checkNotNull(options); + Preconditions.checkNotNull(windowingStrategy); + Preconditions.checkNotNull(doFn); + + this.doFn = doFn; + this.options = options; + this.windowingStrategy = windowingStrategy; + } + +// protected void writeObject(ObjectOutputStream out) +// throws IOException, ClassNotFoundException { +// out.defaultWriteObject(); +// ObjectMapper mapper = new ObjectMapper(); +// mapper.writeValue(out, options); +// } +// +// protected void readObject(ObjectInputStream in) +// throws IOException, ClassNotFoundException { +// in.defaultReadObject(); +// ObjectMapper mapper = new ObjectMapper(); +// options = mapper.readValue(in, PipelineOptions.class); +// } + + private void initContext(DoFn function, Collector> outCollector) { + if (this.context == null) { + this.context = new DoFnProcessContext(function, outCollector); + } + } + + @Override + public void flatMap(WindowedValue value, Collector> out) throws Exception { + this.initContext(doFn, out); + + // for each window the element belongs to, create a new copy here. + Collection windows = value.getWindows(); + if (windows.size() <= 1) { + processElement(value); + } else { + for (BoundedWindow window : windows) { + processElement(WindowedValue.of( + value.getValue(), value.getTimestamp(), window, value.getPane())); + } + } + } + + private void processElement(WindowedValue value) throws Exception { + this.context.setElement(value); + this.doFn.startBundle(context); + doFn.processElement(context); + this.doFn.finishBundle(context); + } + + private class DoFnProcessContext extends DoFn.ProcessContext { + + private final DoFn fn; + + protected final Collector> collector; + + private WindowedValue element; + + private DoFnProcessContext(DoFn function, Collector> outCollector) { + function.super(); + super.setupDelegateAggregators(); + + this.fn = function; + this.collector = outCollector; + } + + public void setElement(WindowedValue value) { + this.element = value; + } + + @Override + public IN element() { + return this.element.getValue(); + } + + @Override + public Instant timestamp() { + return this.element.getTimestamp(); + } + + @Override + public BoundedWindow window() { +// if (!(fn instanceof DoFn.RequiresWindowAccess)) { +// throw new UnsupportedOperationException( +// "window() is only available in the context of a DoFn marked as RequiresWindow."); +// } + + Collection windows = this.element.getWindows(); + if (windows.size() != 1) { + throw new IllegalArgumentException("Each element is expected to belong to 1 window. " + + "This belongs to " + windows.size() + "."); + } + return windows.iterator().next(); + } + + @Override + public PaneInfo pane() { + return this.element.getPane(); + } + + @Override + public WindowingInternals windowingInternals() { + return windowingInternalsHelper(element, collector); + } + + @Override + public PipelineOptions getPipelineOptions() { + return options; + } + + @Override + public T sideInput(PCollectionView view) { + throw new RuntimeException("sideInput() is not supported in Streaming mode."); + } + + @Override + public void output(OUTDF output) { + outputWithTimestamp(output, this.element.getTimestamp()); + } + + @Override + public void outputWithTimestamp(OUTDF output, Instant timestamp) { + outputWithTimestampHelper(element, output, timestamp, collector); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + sideOutputWithTimestamp(tag, output, this.element.getTimestamp()); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + sideOutputWithTimestampHelper(element, output, timestamp, collector, tag); + } + + @Override + protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { + Accumulator acc = getRuntimeContext().getAccumulator(name); + if (acc != null) { + AccumulatorHelper.compareAccumulatorTypes(name, + SerializableFnAggregatorWrapper.class, acc.getClass()); + return (Aggregator) acc; + } + + SerializableFnAggregatorWrapper accumulator = + new SerializableFnAggregatorWrapper<>(combiner); + getRuntimeContext().addAccumulator(name, accumulator); + return accumulator; + } + } + + protected void checkTimestamp(WindowedValue ref, Instant timestamp) { + if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) { + throw new IllegalArgumentException(String.format( + "Cannot output with timestamp %s. Output timestamps must be no earlier than the " + + "timestamp of the current input (%s) minus the allowed skew (%s). See the " + + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.", + timestamp, ref.getTimestamp(), + PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod()))); + } + } + + protected WindowedValue makeWindowedValue( + T output, Instant timestamp, Collection windows, PaneInfo pane) { + final Instant inputTimestamp = timestamp; + final WindowFn windowFn = windowingStrategy.getWindowFn(); + + if (timestamp == null) { + timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + if (windows == null) { + try { + windows = windowFn.assignWindows(windowFn.new AssignContext() { + @Override + public Object element() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input element when none was available"); // TODO: 12/16/15 aljoscha's comment in slack + } + + @Override + public Instant timestamp() { + if (inputTimestamp == null) { + throw new UnsupportedOperationException( + "WindowFn attempted to access input timestamp when none was available"); + } + return inputTimestamp; + } + + @Override + public Collection windows() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input windows when none were available"); + } + }); + } catch (Exception e) { + Throwables.propagateIfInstanceOf(e, UserCodeException.class); + throw new UserCodeException(e); + } + } + + return WindowedValue.of(output, timestamp, windows, pane); + } + + /////////// ABSTRACT METHODS TO BE IMPLEMENTED BY SUBCLASSES ///////////////// + + public abstract void outputWithTimestampHelper( + WindowedValue inElement, + OUTDF output, + Instant timestamp, + Collector> outCollector); + + public abstract void sideOutputWithTimestampHelper( + WindowedValue inElement, + T output, + Instant timestamp, + Collector> outCollector, + TupleTag tag); + + public abstract WindowingInternals windowingInternalsHelper( + WindowedValue inElement, + Collector> outCollector); + +} \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java new file mode 100644 index 000000000000..c52fabea533f --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -0,0 +1,601 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming; + +import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; +import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.*; +import com.google.cloud.dataflow.sdk.coders.*; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import com.google.cloud.dataflow.sdk.transforms.Aggregator; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; +import com.google.cloud.dataflow.sdk.util.*; +import com.google.cloud.dataflow.sdk.values.*; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.accumulators.AccumulatorHelper; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.KeyedStream; +import org.apache.flink.streaming.api.operators.*; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTaskState; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.*; + +/** + * This class is the key class implementing all the windowing/triggering logic of Google Dataflow. + * To provide full compatibility and support all the windowing/triggering combinations offered by + * Datadlow, we opted for a strategy that uses the SDK's code for doing these operations + * ({@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn}. + *

    + * In a nutshell, when the execution arrives to this operator, we expect to have a stream already + * grouped by key. Each of the elements that enter here, registers a timer + * (see {@link TimerInternals#setTimer(TimerInternals.TimerData)} in the + * {@link FlinkGroupAlsoByWindowWrapper#activeTimers}. + * This is essentially a timestamp indicating when to trigger the computation over the window this + * element belongs to. + *

    + * When a watermark arrives, all the registered timers are checked to see which ones are ready to + * fire (see {@link FlinkGroupAlsoByWindowWrapper#processWatermark(Watermark)}). These are deregistered from + * the {@link FlinkGroupAlsoByWindowWrapper#activeTimers} + * list, and are fed into the {@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn} + * for furhter processing. + */ +public class FlinkGroupAlsoByWindowWrapper + extends AbstractStreamOperator>> + implements OneInputStreamOperator>, WindowedValue>> { + + private static final long serialVersionUID = 1L; + + private transient PipelineOptions options; + + private transient CoderRegistry coderRegistry; + + private StreamingGroupAlsoByWindowsDoFn operator; + + private ProcessContext context; + + private final WindowingStrategy windowingStrategy; + + private final Combine.KeyedCombineFn combineFn; + + private final KvCoder inputKvCoder; + + /** + * State is kept per-key. This data structure keeps this mapping between an active key, i.e. a + * key whose elements are currently waiting to be processed, and its associated state. + */ + private Map> perKeyStateInternals = new HashMap<>(); + + /** + * Timers waiting to be processed. + */ + private Map> activeTimers = new HashMap<>(); + + private FlinkTimerInternals timerInternals = new FlinkTimerInternals(); + + /** + * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy. + * This method assumes that elements are already grouped by key. + *

    + * The difference with {@link #createForIterable(PipelineOptions, PCollection, KeyedStream)} + * is that this method assumes that a combiner function is provided + * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}). + * A combiner helps at increasing the speed and, in most of the cases, reduce the per-window state. + * + * @param options the general job configuration options. + * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}. + * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key. + * @param combiner the combiner to be used. + * @param outputKvCoder the type of the output values. + */ + public static DataStream>> create( + PipelineOptions options, + PCollection input, + KeyedStream>, K> groupedStreamByKey, + Combine.KeyedCombineFn combiner, + KvCoder outputKvCoder) { + + KvCoder inputKvCoder = (KvCoder) input.getCoder(); + FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper<>(options, + input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, combiner); + + Coder>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of( + outputKvCoder, + input.getWindowingStrategy().getWindowFn().windowCoder()); + + CoderTypeInformation>> outputTypeInfo = + new CoderTypeInformation<>(windowedOutputElemCoder); + + DataStream>> groupedByKeyAndWindow = groupedStreamByKey + .transform("GroupByWindowWithCombiner", + new CoderTypeInformation<>(outputKvCoder), + windower) + .returns(outputTypeInfo); + + return groupedByKeyAndWindow; + } + + /** + * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy. + * This method assumes that elements are already grouped by key. + *

    + * The difference with {@link #create(PipelineOptions, PCollection, KeyedStream, Combine.KeyedCombineFn, KvCoder)} + * is that this method assumes no combiner function + * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}). + * + * @param options the general job configuration options. + * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}. + * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key. + */ + public static DataStream>>> createForIterable( + PipelineOptions options, + PCollection input, + KeyedStream>, K> groupedStreamByKey) { + + KvCoder inputKvCoder = (KvCoder) input.getCoder(); + Coder keyCoder = inputKvCoder.getKeyCoder(); + Coder inputValueCoder = inputKvCoder.getValueCoder(); + + FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper(options, + input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, null); + + Coder> valueIterCoder = IterableCoder.of(inputValueCoder); + KvCoder> outputElemCoder = KvCoder.of(keyCoder, valueIterCoder); + + Coder>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of( + outputElemCoder, + input.getWindowingStrategy().getWindowFn().windowCoder()); + + CoderTypeInformation>>> outputTypeInfo = + new CoderTypeInformation<>(windowedOutputElemCoder); + + DataStream>>> groupedByKeyAndWindow = groupedStreamByKey + .transform("GroupByWindow", + new CoderTypeInformation<>(windowedOutputElemCoder), + windower) + .returns(outputTypeInfo); + + return groupedByKeyAndWindow; + } + + public static FlinkGroupAlsoByWindowWrapper createForTesting(PipelineOptions options, + CoderRegistry registry, + WindowingStrategy windowingStrategy, + KvCoder inputCoder, + Combine.KeyedCombineFn combiner) { + return new FlinkGroupAlsoByWindowWrapper(options, registry, windowingStrategy, inputCoder, combiner); + } + + private FlinkGroupAlsoByWindowWrapper(PipelineOptions options, + CoderRegistry registry, + WindowingStrategy windowingStrategy, + KvCoder inputCoder, + Combine.KeyedCombineFn combiner) { + + this.options = Preconditions.checkNotNull(options); + this.coderRegistry = Preconditions.checkNotNull(registry); + this.inputKvCoder = Preconditions.checkNotNull(inputCoder);//(KvCoder) input.getCoder(); + this.combineFn = combiner; + this.windowingStrategy = Preconditions.checkNotNull(windowingStrategy);//input.getWindowingStrategy(); + this.operator = createGroupAlsoByWindowOperator(); + this.chainingStrategy = ChainingStrategy.ALWAYS; + } + + @Override + public void open() throws Exception { + super.open(); + this.context = new ProcessContext(operator, new TimestampedCollector<>(output), this.timerInternals); + + // this is to cover the case that this is the state after a recovery. + // In this case, the restoreState() has already initialized the timerInternals to a certain value. + TimerOrElement>> element = this.timerInternals.getElement(); + if (element != null) { + if (element.isTimer()) { + throw new RuntimeException("The recovered element cannot be a Timer."); + } + K key = element.element().getValue().getKey(); + FlinkStateInternals stateForKey = getStateInternalsForKey(key); + this.context.setElement(element, stateForKey); + } + } + + /** + * Create the adequate {@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn}, + * if not already created. + * If a {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn} was provided, then + * a function with that combiner is created, so that elements are combined as they arrive. This is + * done for speed and (in most of the cases) for reduction of the per-window state. + */ + private StreamingGroupAlsoByWindowsDoFn createGroupAlsoByWindowOperator() { + if (this.operator == null) { + if (this.combineFn == null) { + Coder inputValueCoder = inputKvCoder.getValueCoder(); + + this.operator = StreamingGroupAlsoByWindowsDoFn.createForIterable( + this.windowingStrategy, inputValueCoder); + } else { + + Coder inputKeyCoder = inputKvCoder.getKeyCoder(); + //CoderRegistry dataflowRegistry = input.getPipeline().getCoderRegistry(); + + AppliedCombineFn appliedCombineFn = AppliedCombineFn + .withInputCoder(combineFn, coderRegistry, inputKvCoder); + + this.operator = StreamingGroupAlsoByWindowsDoFn.create( + this.windowingStrategy, appliedCombineFn, inputKeyCoder); + } + } + return this.operator; + } + + + @Override + public void processElement(StreamRecord>> element) throws Exception { + WindowedValue> value = element.getValue(); + TimerOrElement>> elem = TimerOrElement.element(value); + processElementOrTimer(elem); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + + context.setCurrentWatermark(new Instant(mark.getTimestamp())); + + Set toFire = getTimersReadyToProcess(mark.getTimestamp()); + if (!toFire.isEmpty()) { + for (TimerOrElement timer : toFire) { + processElementOrTimer(timer); + } + } + + /** + * This is to take into account the different semantics of the Watermark in Flink and + * in Dataflow. To understand the reasoning behind the Dataflow semantics and its + * watermark holding logic, see the documentation of + * {@link WatermarkHold#addHold(ReduceFn.ProcessValueContext, boolean)} + * */ + long millis = Long.MAX_VALUE; + for (FlinkStateInternals state : perKeyStateInternals.values()) { + Instant watermarkHold = state.getWatermarkHold(); + if (watermarkHold != null && watermarkHold.getMillis() < millis) { + millis = watermarkHold.getMillis(); + } + } + + if (mark.getTimestamp() < millis) { + millis = mark.getTimestamp(); + } + + // Don't forget to re-emit the watermark for further operators down the line. + // This is critical for jobs with multiple aggregation steps. + // Imagine a job with a groupByKey() on key K1, followed by a map() that changes + // the key K1 to K2, and another groupByKey() on K2. In this case, if the watermark + // is not re-emitted, the second aggregation would never be triggered, and no result + // will be produced. + output.emitWatermark(new Watermark(millis)); + } + + private void processElementOrTimer(TimerOrElement>> timerOrElement) throws Exception { + K key = timerOrElement.isTimer() ? + (K) timerOrElement.key() : + timerOrElement.element().getValue().getKey(); + + context.setElement(timerOrElement, getStateInternalsForKey(key)); + + operator.startBundle(context); + operator.processElement(context); + operator.finishBundle(context); + } + + private void registerActiveTimer(K key, TimerInternals.TimerData timer) { + Set timersForKey = activeTimers.get(key); + if (timersForKey == null) { + timersForKey = new HashSet<>(); + } + timersForKey.add(timer); + activeTimers.put(key, timersForKey); + } + + private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) { + Set timersForKey = activeTimers.get(key); + if (timersForKey != null) { + timersForKey.remove(timer); + if (timersForKey.isEmpty()) { + activeTimers.remove(key); + } else { + activeTimers.put(key, timersForKey); + } + } + } + + /** + * Returns the list of timers that are ready to fire. These are the timers + * that are registered to be triggered at a time before the current watermark. + * We keep these timers in a Set, so that they are deduplicated, as the same + * timer can be registered multiple times. + */ + private Set getTimersReadyToProcess(long currentWatermark) { + + // we keep the timers to return in a different list and launch them later + // because we cannot prevent a trigger from registering another trigger, + // which would lead to concurrent modification exception. + Set toFire = new HashSet<>(); + + Iterator>> it = activeTimers.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry> keyWithTimers = it.next(); + + Iterator timerIt = keyWithTimers.getValue().iterator(); + while (timerIt.hasNext()) { + TimerInternals.TimerData timerData = timerIt.next(); + if (timerData.getTimestamp().isBefore(currentWatermark)) { + TimerOrElement timer = TimerOrElement.timer(keyWithTimers.getKey(), timerData); + toFire.add(timer); + timerIt.remove(); + } + } + + if (keyWithTimers.getValue().isEmpty()) { + it.remove(); + } + } + return toFire; + } + + /** + * Gets the state associated with the specified key. + * + * @param key the key whose state we want. + * @return The {@link FlinkStateInternals} + * associated with that key. + */ + private FlinkStateInternals getStateInternalsForKey(K key) { + FlinkStateInternals stateInternals = perKeyStateInternals.get(key); + if (stateInternals == null) { + Coder windowCoder = this.windowingStrategy.getWindowFn().windowCoder(); + stateInternals = new FlinkStateInternals<>(key, inputKvCoder.getKeyCoder(), windowCoder, combineFn); + perKeyStateInternals.put(key, stateInternals); + } + return stateInternals; + } + + private class FlinkTimerInternals extends AbstractFlinkTimerInternals { + + @Override + protected void registerTimer(K key, TimerData timerKey) { + registerActiveTimer(key, timerKey); + } + + @Override + protected void unregisterTimer(K key, TimerData timerKey) { + unregisterActiveTimer(key, timerKey); + } + } + + private class ProcessContext extends DoFn>>, KV>.ProcessContext { + + private final FlinkTimerInternals timerInternals; + + private final DoFn>>, KV> fn; + + private final Collector>> collector; + + private FlinkStateInternals stateInternals; + + private TimerOrElement>> element; + + public ProcessContext(DoFn>>, KV> function, + Collector>> outCollector, + FlinkTimerInternals timerInternals) { + function.super(); + super.setupDelegateAggregators(); + + this.fn = Preconditions.checkNotNull(function); + this.collector = Preconditions.checkNotNull(outCollector); + this.timerInternals = Preconditions.checkNotNull(timerInternals); + } + + public void setElement(TimerOrElement>> value, + FlinkStateInternals stateForKey) { + this.element = value; + this.stateInternals = stateForKey; + this.timerInternals.setElement(value); + } + + public void setCurrentWatermark(Instant watermark) { + this.timerInternals.setCurrentWatermark(watermark); + } + + @Override + public TimerOrElement element() { + if (element != null && !this.element.isTimer()) { + return TimerOrElement.element(this.element.element().getValue()); + } + return this.element; + } + + @Override + public Instant timestamp() { + return this.element.isTimer() ? + this.element.getTimer().getTimestamp() : + this.element.element().getTimestamp(); + } + + @Override + public PipelineOptions getPipelineOptions() { + return options; + } + + @Override + public void output(KV output) { + throw new UnsupportedOperationException( + "output() is not available when grouping by window."); + } + + @Override + public void outputWithTimestamp(KV output, Instant timestamp) { + throw new UnsupportedOperationException( + "outputWithTimestamp() is not available when grouping by window."); + } + + @Override + public PaneInfo pane() { + return this.element.element().getPane(); + } + + @Override + public BoundedWindow window() { + if (!(fn instanceof DoFn.RequiresWindowAccess)) { + throw new UnsupportedOperationException( + "window() is only available in the context of a DoFn marked as RequiresWindow."); + } + + Collection windows = this.element.element().getWindows(); + if (windows.size() != 1) { + throw new IllegalArgumentException("Each element is expected to belong to 1 window. " + + "This belongs to " + windows.size() + "."); + } + return windows.iterator().next(); + } + + @Override + public WindowingInternals>>, KV> windowingInternals() { + return new WindowingInternals>>, KV>() { + + @Override + public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() { + return stateInternals; + } + + @Override + public void outputWindowedValue(KV output, Instant timestamp, Collection windows, PaneInfo pane) { + collector.collect(WindowedValue.of(output, timestamp, windows, pane)); + } + + @Override + public TimerInternals timerInternals() { + return timerInternals; + } + + @Override + public Collection windows() { + return element.element().getWindows(); + } + + @Override + public PaneInfo pane() { + return element.element().getPane(); + } + + @Override + public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { + throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode."); + } + }; + } + + @Override + public T sideInput(PCollectionView view) { + throw new RuntimeException("sideInput() is not supported in Streaming mode."); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + // ignore the side output, this can happen when a user does not register + // side outputs but then outputs using a freshly created TupleTag. + throw new RuntimeException("sideOutput() is not available when grouping by window."); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + sideOutput(tag, output); + } + + @Override + protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { + Accumulator acc = getRuntimeContext().getAccumulator(name); + if (acc != null) { + AccumulatorHelper.compareAccumulatorTypes(name, + SerializableFnAggregatorWrapper.class, acc.getClass()); + return (Aggregator) acc; + } + + SerializableFnAggregatorWrapper accumulator = + new SerializableFnAggregatorWrapper<>(combiner); + getRuntimeContext().addAccumulator(name, accumulator); + return accumulator; + } + } + + ////////////// Checkpointing implementation //////////////// + + @Override + public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception { + StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp); + StateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp); + StateCheckpointWriter writer = StateCheckpointWriter.create(out); + Coder keyCoder = inputKvCoder.getKeyCoder(); + + // checkpoint the timers + StateCheckpointUtils.encodeTimers(activeTimers, writer, keyCoder); + + // checkpoint the state + StateCheckpointUtils.encodeState(perKeyStateInternals, writer, keyCoder); + + // checkpoint the timerInternals + context.timerInternals.encodeTimerInternals(context, writer, + inputKvCoder, windowingStrategy.getWindowFn().windowCoder()); + + taskState.setOperatorState(out.closeAndGetHandle()); + return taskState; + } + + @Override + public void restoreState(StreamTaskState taskState) throws Exception { + super.restoreState(taskState); + + final ClassLoader userClassloader = getUserCodeClassloader(); + + Coder windowCoder = this.windowingStrategy.getWindowFn().windowCoder(); + Coder keyCoder = inputKvCoder.getKeyCoder(); + + @SuppressWarnings("unchecked") + StateHandle inputState = (StateHandle) taskState.getOperatorState(); + DataInputView in = inputState.getState(userClassloader); + StateCheckpointReader reader = new StateCheckpointReader(in); + + // restore the timers + this.activeTimers = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder); + + // restore the state + this.perKeyStateInternals = StateCheckpointUtils.decodeState( + reader, combineFn, keyCoder, windowCoder, userClassloader); + + // restore the timerInternals. + this.timerInternals.restoreTimerInternals(reader, inputKvCoder, windowCoder); + } +} \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java new file mode 100644 index 000000000000..0a0e3012b401 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java @@ -0,0 +1,56 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming; + +import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.util.*; +import com.google.cloud.dataflow.sdk.values.KV; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.KeyedStream; + +public class FlinkGroupByKeyWrapper { + + /** + * Just an auxiliary interface to bypass the fact that java anonymous classes cannot implement + * multiple interfaces. + */ + private interface KeySelectorWithQueryableResultType extends KeySelector>, K>, ResultTypeQueryable { + } + + public static KeyedStream>, K> groupStreamByKey(DataStream>> inputDataStream, KvCoder inputKvCoder) { + final Coder keyCoder = inputKvCoder.getKeyCoder(); + final TypeInformation keyTypeInfo = new CoderTypeInformation<>(keyCoder); + + return inputDataStream.keyBy( + new KeySelectorWithQueryableResultType() { + + @Override + public K getKey(WindowedValue> value) throws Exception { + return value.getValue().getKey(); + } + + @Override + public TypeInformation getProducedType() { + return keyTypeInfo; + } + }); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java new file mode 100644 index 000000000000..200c397dff15 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java @@ -0,0 +1,72 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming; + +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowingInternals; +import com.google.cloud.dataflow.sdk.util.WindowingStrategy; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.util.Map; + +public class FlinkParDoBoundMultiWrapper extends FlinkAbstractParDoWrapper { + + private final TupleTag mainTag; + private final Map, Integer> outputLabels; + + public FlinkParDoBoundMultiWrapper(PipelineOptions options, WindowingStrategy windowingStrategy, DoFn doFn, TupleTag mainTag, Map, Integer> tagsToLabels) { + super(options, windowingStrategy, doFn); + this.mainTag = Preconditions.checkNotNull(mainTag); + this.outputLabels = Preconditions.checkNotNull(tagsToLabels); + } + + @Override + public void outputWithTimestampHelper(WindowedValue inElement, OUT output, Instant timestamp, Collector> collector) { + checkTimestamp(inElement, timestamp); + Integer index = outputLabels.get(mainTag); + collector.collect(makeWindowedValue( + new RawUnionValue(index, output), + timestamp, + inElement.getWindows(), + inElement.getPane())); + } + + @Override + public void sideOutputWithTimestampHelper(WindowedValue inElement, T output, Instant timestamp, Collector> collector, TupleTag tag) { + checkTimestamp(inElement, timestamp); + Integer index = outputLabels.get(tag); + if (index != null) { + collector.collect(makeWindowedValue( + new RawUnionValue(index, output), + timestamp, + inElement.getWindows(), + inElement.getPane())); + } + } + + @Override + public WindowingInternals windowingInternalsHelper(WindowedValue inElement, Collector> outCollector) { + throw new RuntimeException("FlinkParDoBoundMultiWrapper is just an internal operator serving as " + + "an intermediate transformation for the ParDo.BoundMulti translation. windowingInternals() " + + "is not available in this class."); + } +} \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java new file mode 100644 index 000000000000..18d42494fc04 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java @@ -0,0 +1,89 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; +import com.google.cloud.dataflow.sdk.util.*; +import com.google.cloud.dataflow.sdk.util.state.StateInternals; +import com.google.cloud.dataflow.sdk.values.TupleTag; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.*; + +public class FlinkParDoBoundWrapper extends FlinkAbstractParDoWrapper { + + public FlinkParDoBoundWrapper(PipelineOptions options, WindowingStrategy windowingStrategy, DoFn doFn) { + super(options, windowingStrategy, doFn); + } + + @Override + public void outputWithTimestampHelper(WindowedValue inElement, OUT output, Instant timestamp, Collector> collector) { + checkTimestamp(inElement, timestamp); + collector.collect(makeWindowedValue( + output, + timestamp, + inElement.getWindows(), + inElement.getPane())); + } + + @Override + public void sideOutputWithTimestampHelper(WindowedValue inElement, T output, Instant timestamp, Collector> outCollector, TupleTag tag) { + // ignore the side output, this can happen when a user does not register + // side outputs but then outputs using a freshly created TupleTag. + throw new RuntimeException("sideOutput() not not available in ParDo.Bound()."); + } + + @Override + public WindowingInternals windowingInternalsHelper(final WindowedValue inElement, final Collector> collector) { + return new WindowingInternals() { + @Override + public StateInternals stateInternals() { + throw new NullPointerException("StateInternals are not available for ParDo.Bound()."); + } + + @Override + public void outputWindowedValue(OUT output, Instant timestamp, Collection windows, PaneInfo pane) { + collector.collect(makeWindowedValue(output, timestamp, windows, pane)); + } + + @Override + public TimerInternals timerInternals() { + throw new NullPointerException("TimeInternals are not available for ParDo.Bound()."); + } + + @Override + public Collection windows() { + return inElement.getWindows(); + } + + @Override + public PaneInfo pane() { + return inElement.getPane(); + } + + @Override + public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { + throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode."); + } + }; + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java new file mode 100644 index 000000000000..17e0746fdbae --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java @@ -0,0 +1,76 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.io; + +import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.UnboundedSource; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; + +import javax.annotation.Nullable; +import java.util.List; + +public class UnboundedFlinkSource extends UnboundedSource { + + private final PipelineOptions options; + private final RichParallelSourceFunction flinkSource; + + public UnboundedFlinkSource(PipelineOptions pipelineOptions, RichParallelSourceFunction source) { + if(!pipelineOptions.getRunner().equals(FlinkPipelineRunner.class)) { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } + options = Preconditions.checkNotNull(pipelineOptions); + flinkSource = Preconditions.checkNotNull(source); + validate(); + } + + public RichParallelSourceFunction getFlinkSource() { + return this.flinkSource; + } + + @Override + public List> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } + + @Override + public UnboundedReader createReader(PipelineOptions options, @Nullable C checkpointMark) { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } + + @Nullable + @Override + public Coder getCheckpointMarkCoder() { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } + + + @Override + public void validate() { + Preconditions.checkNotNull(options); + Preconditions.checkNotNull(flinkSource); + if(!options.getRunner().equals(FlinkPipelineRunner.class)) { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } + } + + @Override + public Coder getDefaultOutputCoder() { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java new file mode 100644 index 000000000000..2b0d6dc4a866 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java @@ -0,0 +1,228 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.io; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.io.UnboundedSource; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.Serializable; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; + +import static com.google.common.base.Preconditions.checkArgument; + +public class UnboundedSocketSource extends UnboundedSource { + + private static final Coder DEFAULT_SOCKET_CODER = StringUtf8Coder.of(); + + private static final long serialVersionUID = 1L; + + private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500; + + private static final int CONNECTION_TIMEOUT_TIME = 0; + + private final String hostname; + private final int port; + private final char delimiter; + private final long maxNumRetries; + private final long delayBetweenRetries; + + public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries) { + this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP); + } + + public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) { + this.hostname = hostname; + this.port = port; + this.delimiter = delimiter; + this.maxNumRetries = maxNumRetries; + this.delayBetweenRetries = delayBetweenRetries; + } + + public String getHostname() { + return this.hostname; + } + + public int getPort() { + return this.port; + } + + public char getDelimiter() { + return this.delimiter; + } + + public long getMaxNumRetries() { + return this.maxNumRetries; + } + + public long getDelayBetweenRetries() { + return this.delayBetweenRetries; + } + + @Override + public List> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception { + return Collections.>singletonList(this); + } + + @Override + public UnboundedReader createReader(PipelineOptions options, @Nullable C checkpointMark) { + return new UnboundedSocketReader(this); + } + + @Nullable + @Override + public Coder getCheckpointMarkCoder() { + // Flink and Dataflow have different checkpointing mechanisms. + // In our case we do not need a coder. + return null; + } + + @Override + public void validate() { + checkArgument(port > 0 && port < 65536, "port is out of range"); + checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)"); + checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive"); + } + + @Override + public Coder getDefaultOutputCoder() { + return DEFAULT_SOCKET_CODER; + } + + public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader implements Serializable { + + private static final long serialVersionUID = 7526472295622776147L; + private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class); + + private final UnboundedSocketSource source; + + private Socket socket; + private BufferedReader reader; + + private boolean isRunning; + + private String currentRecord; + + public UnboundedSocketReader(UnboundedSocketSource source) { + this.source = source; + } + + private void openConnection() throws IOException { + this.socket = new Socket(); + this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME); + this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream())); + this.isRunning = true; + } + + @Override + public boolean start() throws IOException { + int attempt = 0; + while (!isRunning) { + try { + openConnection(); + LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort()); + + return advance(); + } catch (IOException e) { + LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs..."); + + if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) { + try { + Thread.sleep(this.source.getDelayBetweenRetries()); + } catch (InterruptedException e1) { + e1.printStackTrace(); + } + } else { + this.isRunning = false; + break; + } + } + } + LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort()); + return false; + } + + @Override + public boolean advance() throws IOException { + final StringBuilder buffer = new StringBuilder(); + int data; + while (isRunning && (data = reader.read()) != -1) { + // check if the string is complete + if (data != this.source.getDelimiter()) { + buffer.append((char) data); + } else { + if (buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') { + buffer.setLength(buffer.length() - 1); + } + this.currentRecord = buffer.toString(); + buffer.setLength(0); + return true; + } + } + return false; + } + + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + return new byte[0]; + } + + @Override + public String getCurrent() throws NoSuchElementException { + return this.currentRecord; + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return Instant.now(); + } + + @Override + public void close() throws IOException { + this.reader.close(); + this.socket.close(); + this.isRunning = false; + LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + "."); + } + + @Override + public Instant getWatermark() { + return Instant.now(); + } + + @Override + public CheckpointMark getCheckpointMark() { + return null; + } + + @Override + public UnboundedSource getCurrentSource() { + return this.source; + } + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java new file mode 100644 index 000000000000..3e248a662597 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -0,0 +1,120 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.io; + +import com.google.cloud.dataflow.sdk.io.Read; +import com.google.cloud.dataflow.sdk.io.UnboundedSource; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.operators.Triggerable; +import org.joda.time.Instant; + +import java.util.Collection; + +public class UnboundedSourceWrapper extends RichSourceFunction> implements EventTimeSourceFunction>, Triggerable { + + private final String name; + private final UnboundedSource.UnboundedReader reader; + + private StreamingRuntimeContext runtime = null; + private StreamSource.ManualWatermarkContext context = null; + + private volatile boolean isRunning = false; + + public UnboundedSourceWrapper(PipelineOptions options, Read.Unbounded transform) { + this.name = transform.getName(); + this.reader = transform.getSource().createReader(options, null); + } + + public String getName() { + return this.name; + } + + WindowedValue makeWindowedValue( + T output, Instant timestamp, Collection windows, PaneInfo pane) { + if (timestamp == null) { + timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + return WindowedValue.of(output, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); + } + + @Override + public void run(SourceContext> ctx) throws Exception { + if (!(ctx instanceof StreamSource.ManualWatermarkContext)) { + throw new RuntimeException("We assume that all sources in Dataflow are EventTimeSourceFunction. " + + "Apparently " + this.name + " is not. Probably you should consider writing your own Wrapper for this source."); + } + + context = (StreamSource.ManualWatermarkContext) ctx; + runtime = (StreamingRuntimeContext) getRuntimeContext(); + + this.isRunning = reader.start(); + setNextWatermarkTimer(this.runtime); + + while (isRunning) { + + // get it and its timestamp from the source + T item = reader.getCurrent(); + Instant timestamp = reader.getCurrentTimestamp(); + + long milliseconds = timestamp.getMillis(); + + // write it to the output collector + synchronized (ctx.getCheckpointLock()) { + ctx.collectWithTimestamp(makeWindowedValue(item, timestamp, null, PaneInfo.NO_FIRING), milliseconds); + } + + // try to go to the next record + this.isRunning = reader.advance(); + } + } + + @Override + public void cancel() { + isRunning = false; + } + + @Override + public void trigger(long timestamp) throws Exception { + if (this.isRunning) { + synchronized (context.getCheckpointLock()) { + long watermarkMillis = this.reader.getWatermark().getMillis(); + context.emitWatermark(new Watermark(watermarkMillis)); + } + setNextWatermarkTimer(this.runtime); + } + } + + private void setNextWatermarkTimer(StreamingRuntimeContext runtime) { + if (this.isRunning) { + long watermarkInterval = runtime.getExecutionConfig().getAutoWatermarkInterval(); + long timeToNextWatermark = getTimeToNextWaternark(watermarkInterval); + runtime.registerTimer(timeToNextWatermark, this); + } + } + + private long getTimeToNextWaternark(long watermarkInterval) { + return System.currentTimeMillis() + watermarkInterval; + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java new file mode 100644 index 000000000000..4401eb372f5c --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java @@ -0,0 +1,139 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; + +import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.TimerInternals; +import com.google.cloud.dataflow.sdk.util.TimerOrElement; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.values.KV; +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.Serializable; + +public abstract class AbstractFlinkTimerInternals implements TimerInternals, Serializable { + + private TimerOrElement>> element; + + private Instant currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + + public TimerOrElement>> getElement() { + return this.element; + } + + public void setElement(TimerOrElement>> value) { + this.element = value; + } + + public void setCurrentWatermark(Instant watermark) { + checkIfValidWatermark(watermark); + this.currentWatermark = watermark; + } + + private void setCurrentWatermarkAfterRecovery(Instant watermark) { + if(!currentWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) { + throw new RuntimeException("Explicitly setting the watermark is only allowed on " + + "initialization after recovery from a node failure. Apparently this is not " + + "the case here as the watermark is already set."); + } + this.currentWatermark = watermark; + } + + @Override + public void setTimer(com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData timerKey) { + K key = element.isTimer() ? (K) element.key() : element.element().getValue().getKey(); + registerTimer(key, timerKey); + } + + protected abstract void registerTimer(K key, TimerData timerKey); + + @Override + public void deleteTimer(com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData timerKey) { + K key = element.isTimer() ? (K) element.key() : element.element().getValue().getKey(); + unregisterTimer(key, timerKey); + } + + protected abstract void unregisterTimer(K key, TimerData timerKey); + + @Override + public Instant currentProcessingTime() { + return Instant.now(); + } + + @Override + public Instant currentWatermarkTime() { + return this.currentWatermark; + } + + private void checkIfValidWatermark(Instant newWatermark) { + if (currentWatermark.isAfter(newWatermark)) { + throw new IllegalArgumentException(String.format( + "Cannot set current watermark to %s. Newer watermarks " + + "must be no earlier than the current one (%s).", + newWatermark, this.currentWatermark)); + } + } + + public void encodeTimerInternals(DoFn.ProcessContext context, + StateCheckpointWriter writer, + KvCoder kvCoder, + Coder windowCoder) throws IOException { + if (context == null) { + throw new RuntimeException("The Context has not been initialized."); + } + + if (element != null && !element.isTimer()) { + // create the element coder + WindowedValue.WindowedValueCoder> elementCoder = WindowedValue + .getFullCoder(kvCoder, windowCoder); + + CoderTypeSerializer>> serializer = + new CoderTypeSerializer<>(elementCoder); + + writer.writeByte((byte) 1); + writer.serializeObject(element.element(), serializer); + } else { + // just setting a flag to 0, meaning that there is no value. + writer.writeByte((byte) 0); + } + writer.setTimestamp(currentWatermark); + } + + public void restoreTimerInternals(StateCheckpointReader reader, + KvCoder kvCoder, + Coder windowCoder) throws IOException { + + boolean isSet = (reader.getByte() == (byte) 1); + if (!isSet) { + this.element = null; + } else { + WindowedValue.WindowedValueCoder> elementCoder = WindowedValue + .getFullCoder(kvCoder, windowCoder); + + CoderTypeSerializer>> serializer = + new CoderTypeSerializer<>(elementCoder); + + WindowedValue> elem = reader.deserializeObject(serializer); + this.element = TimerOrElement.element(elem); + } + setCurrentWatermarkAfterRecovery(reader.getTimestamp()); + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java new file mode 100644 index 000000000000..03b8bb5be4c0 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java @@ -0,0 +1,533 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; + +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.state.*; +import com.google.protobuf.ByteString; +import org.apache.flink.util.InstantiationUtil; +import org.joda.time.Instant; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.*; + +public class FlinkStateInternals extends MergingStateInternals { + + private final K key; + + private final Coder keyCoder; + + private final Combine.KeyedCombineFn combineFn; + + private final Coder windowCoder; + + private Instant watermarkHoldAccessor; + + public FlinkStateInternals(K key, + Coder keyCoder, + Coder windowCoder, + Combine.KeyedCombineFn combineFn) { + this.key = key; + this.combineFn = combineFn; + this.windowCoder = windowCoder; + this.keyCoder = keyCoder; + } + + public Instant getWatermarkHold() { + return watermarkHoldAccessor; + } + + /** + * This is the interface state has to implement in order for it to be fault tolerant when + * executed by the FlinkPipelineRunner. + */ + private interface CheckpointableIF { + + boolean shouldPersist(); + + void persistState(StateCheckpointWriter checkpointBuilder) throws IOException; + } + + protected final StateTable inMemoryState = new StateTable() { + + @Override + protected StateTag.StateBinder binderForNamespace(final StateNamespace namespace) { + return new StateTag.StateBinder() { + + @Override + public ValueState bindValue(StateTag> address, Coder coder) { + return new FlinkInMemoryValue<>(encodeKey(namespace, address), coder); + } + + @Override + public BagState bindBag(StateTag> address, Coder elemCoder) { + return new FlinkInMemoryBag<>(encodeKey(namespace, address), elemCoder); + } + + @Override + public CombiningValueStateInternal bindCombiningValue( + StateTag> address, + Coder accumCoder, Combine.CombineFn combineFn) { + return new FlinkInMemoryCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder); + } + + @Override + public WatermarkStateInternal bindWatermark(StateTag address) { + return new FlinkWatermarkStateInternalImpl(encodeKey(namespace, address)); + } + }; + } + }; + + @Override + public T state(StateNamespace namespace, StateTag address) { + return inMemoryState.get(namespace, address); + } + + public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { + checkpointBuilder.writeInt(getNoOfElements()); + + for (State location : inMemoryState.values()) { + if (!(location instanceof CheckpointableIF)) { + throw new IllegalStateException(String.format( + "%s wasn't created by %s -- unable to persist it", + location.getClass().getSimpleName(), + getClass().getSimpleName())); + } + ((CheckpointableIF) location).persistState(checkpointBuilder); + } + } + + public void restoreState(StateCheckpointReader checkpointReader, ClassLoader loader) + throws IOException, ClassNotFoundException { + + // the number of elements to read. + int noOfElements = checkpointReader.getInt(); + for (int i = 0; i < noOfElements; i++) { + decodeState(checkpointReader, loader); + } + } + + /** + * We remove the first character which encodes the type of the stateTag ('s' for system + * and 'u' for user). For more details check out the source of + * {@link StateTags.StateTagBase#getId()}. + */ + private void decodeState(StateCheckpointReader reader, ClassLoader loader) + throws IOException, ClassNotFoundException { + + StateType stateItemType = StateType.deserialize(reader); + ByteString stateKey = reader.getTag(); + + // first decode the namespace and the tagId... + String[] namespaceAndTag = stateKey.toStringUtf8().split("\\+"); + if (namespaceAndTag.length != 2) { + throw new IllegalArgumentException("Invalid stateKey " + stateKey.toString() + "."); + } + StateNamespace namespace = StateNamespaces.fromString(namespaceAndTag[0], windowCoder); + + // ... decide if it is a system or user stateTag... + char ownerTag = namespaceAndTag[1].charAt(0); + if (ownerTag != 's' && ownerTag != 'u') { + throw new RuntimeException("Invalid StateTag name."); + } + boolean isSystemTag = ownerTag == 's'; + String tagId = namespaceAndTag[1].substring(1); + + // ...then decode the coder (if there is one)... + Coder coder = null; + if (!stateItemType.equals(StateType.WATERMARK)) { + ByteString coderBytes = reader.getData(); + coder = InstantiationUtil.deserializeObject(coderBytes.toByteArray(), loader); + } + + //... and finally, depending on the type of the state being decoded, + // 1) create the adequate stateTag, + // 2) create the state container, + // 3) restore the actual content. + switch (stateItemType) { + case VALUE: { + StateTag stateTag = StateTags.value(tagId, coder); + stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; + FlinkInMemoryValue value = (FlinkInMemoryValue) inMemoryState.get(namespace, stateTag); + value.restoreState(reader); + break; + } + case WATERMARK: { + StateTag stateTag = StateTags.watermarkStateInternal(tagId); + stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; + FlinkWatermarkStateInternalImpl watermark = (FlinkWatermarkStateInternalImpl) inMemoryState.get(namespace, stateTag); + watermark.restoreState(reader); + break; + } + case LIST: { + StateTag stateTag = StateTags.bag(tagId, coder); + stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; + FlinkInMemoryBag bag = (FlinkInMemoryBag) inMemoryState.get(namespace, stateTag); + bag.restoreState(reader); + break; + } + case ACCUMULATOR: { + StateTag stateTag = StateTags.combiningValue(tagId, coder, combineFn.forKey(this.key, keyCoder)); + stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; + FlinkInMemoryCombiningValue combiningValue = (FlinkInMemoryCombiningValue) inMemoryState.get(namespace, stateTag); + combiningValue.restoreState(reader); + break; + } + default: + throw new RuntimeException("Unknown State Type " + stateItemType + "."); + } + } + + private ByteString encodeKey(StateNamespace namespace, StateTag address) { + return ByteString.copyFromUtf8(namespace.stringKey() + "+" + address.getId()); + } + + private int getNoOfElements() { + int noOfElements = 0; + for (State state : inMemoryState.values()) { + if (!(state instanceof CheckpointableIF)) { + throw new RuntimeException("State Implementations used by the " + + "Flink Dataflow Runner should implement the CheckpointableIF interface."); + } + + if (((CheckpointableIF) state).shouldPersist()) { + noOfElements++; + } + } + return noOfElements; + } + + private final class FlinkInMemoryValue implements ValueState, CheckpointableIF { + + private final ByteString stateKey; + private final Coder elemCoder; + + private T value = null; + + public FlinkInMemoryValue(ByteString stateKey, Coder elemCoder) { + this.stateKey = stateKey; + this.elemCoder = elemCoder; + } + + @Override + public void clear() { + value = null; + } + + @Override + public StateContents get() { + return new StateContents() { + @Override + public T read() { + return value; + } + }; + } + + @Override + public void set(T input) { + this.value = input; + } + + @Override + public boolean shouldPersist() { + return value != null; + } + + @Override + public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { + if (value != null) { + + // serialize the coder. + byte[] coder = InstantiationUtil.serializeObject(elemCoder); + + // encode the value into a ByteString + ByteString.Output stream = ByteString.newOutput(); + elemCoder.encode(value, stream, Coder.Context.OUTER); + ByteString data = stream.toByteString(); + + checkpointBuilder.addValueBuilder() + .setTag(stateKey) + .setData(coder) + .setData(data); + } + } + + public void restoreState(StateCheckpointReader checkpointReader) throws IOException { + ByteString valueContent = checkpointReader.getData(); + T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER); + set(outValue); + } + } + + private final class FlinkWatermarkStateInternalImpl + implements WatermarkStateInternal, CheckpointableIF { + + private final ByteString stateKey; + + private Instant minimumHold = null; + + public FlinkWatermarkStateInternalImpl(ByteString stateKey) { + this.stateKey = stateKey; + } + + @Override + public void clear() { + // Even though we're clearing we can't remove this from the in-memory state map, since + // other users may already have a handle on this WatermarkBagInternal. + minimumHold = null; + watermarkHoldAccessor = null; + } + + @Override + public StateContents get() { + return new StateContents() { + @Override + public Instant read() { + return minimumHold; + } + }; + } + + @Override + public void add(Instant watermarkHold) { + if (minimumHold == null || minimumHold.isAfter(watermarkHold)) { + watermarkHoldAccessor = watermarkHold; + minimumHold = watermarkHold; + } + } + + @Override + public StateContents isEmpty() { + return new StateContents() { + @Override + public Boolean read() { + return minimumHold == null; + } + }; + } + + @Override + public String toString() { + return Objects.toString(minimumHold); + } + + @Override + public boolean shouldPersist() { + return minimumHold != null; + } + + @Override + public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { + if (minimumHold != null) { + checkpointBuilder.addWatermarkHoldsBuilder() + .setTag(stateKey) + .setTimestamp(minimumHold); + } + } + + public void restoreState(StateCheckpointReader checkpointReader) throws IOException { + Instant watermark = checkpointReader.getTimestamp(); + add(watermark); + } + } + + private final class FlinkInMemoryCombiningValue + implements CombiningValueStateInternal, CheckpointableIF { + + private final ByteString stateKey; + private final Combine.CombineFn combineFn; + private final Coder accumCoder; + + private AccumT accum; + private boolean isCleared = true; + + private FlinkInMemoryCombiningValue(ByteString stateKey, + Combine.CombineFn combineFn, + Coder accumCoder) { + Preconditions.checkNotNull(combineFn); + Preconditions.checkNotNull(accumCoder); + + this.stateKey = stateKey; + this.combineFn = combineFn; + this.accumCoder = accumCoder; + accum = combineFn.createAccumulator(); + } + + @Override + public void clear() { + accum = combineFn.createAccumulator(); + isCleared = true; + } + + @Override + public StateContents get() { + return new StateContents() { + @Override + public OutputT read() { + return combineFn.extractOutput(accum); + } + }; + } + + @Override + public void add(InputT input) { + isCleared = false; + accum = combineFn.addInput(accum, input); + } + + @Override + public StateContents getAccum() { + return new StateContents() { + @Override + public AccumT read() { + return accum; + } + }; + } + + @Override + public StateContents isEmpty() { + return new StateContents() { + @Override + public Boolean read() { + return isCleared; + } + }; + } + + @Override + public void addAccum(AccumT accum) { + isCleared = false; + this.accum = combineFn.mergeAccumulators(Arrays.asList(this.accum, accum)); + } + + @Override + public boolean shouldPersist() { + return accum != null; + } + + @Override + public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { + if (accum != null) { + + // serialize the coder. + byte[] coder = InstantiationUtil.serializeObject(accumCoder); + + // encode the accumulator into a ByteString + ByteString.Output stream = ByteString.newOutput(); + accumCoder.encode(accum, stream, Coder.Context.OUTER); + ByteString data = stream.toByteString(); + + // put the flag that the next serialized element is an accumulator + checkpointBuilder.addAccumulatorBuilder() + .setTag(stateKey) + .setData(coder) + .setData(data); + } + } + + public void restoreState(StateCheckpointReader checkpointReader) throws IOException { + ByteString valueContent = checkpointReader.getData(); + AccumT accum = this.accumCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER); + addAccum(accum); + } + } + + private static final class FlinkInMemoryBag implements BagState, CheckpointableIF { + private final List contents = new ArrayList<>(); + + private final ByteString stateKey; + private final Coder elemCoder; + + public FlinkInMemoryBag(ByteString stateKey, Coder elemCoder) { + this.stateKey = stateKey; + this.elemCoder = elemCoder; + } + + @Override + public void clear() { + contents.clear(); + } + + @Override + public StateContents> get() { + return new StateContents>() { + @Override + public Iterable read() { + return contents; + } + }; + } + + @Override + public void add(T input) { + contents.add(input); + } + + @Override + public StateContents isEmpty() { + return new StateContents() { + @Override + public Boolean read() { + return contents.isEmpty(); + } + }; + } + + @Override + public boolean shouldPersist() { + return !contents.isEmpty(); + } + + @Override + public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { + if (!contents.isEmpty()) { + // serialize the coder. + byte[] coder = InstantiationUtil.serializeObject(elemCoder); + + checkpointBuilder.addListUpdatesBuilder() + .setTag(stateKey) + .setData(coder) + .writeInt(contents.size()); + + for (T item : contents) { + // encode the element + ByteString.Output stream = ByteString.newOutput(); + elemCoder.encode(item, stream, Coder.Context.OUTER); + ByteString data = stream.toByteString(); + + // add the data to the checkpoint. + checkpointBuilder.setData(data); + } + } + } + + public void restoreState(StateCheckpointReader checkpointReader) throws IOException { + int noOfValues = checkpointReader.getInt(); + for (int j = 0; j < noOfValues; j++) { + ByteString valueContent = checkpointReader.getData(); + T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER); + add(outValue); + } + } + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java new file mode 100644 index 000000000000..ba8ef8962f6c --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java @@ -0,0 +1,89 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; + +import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer; +import com.google.protobuf.ByteString; +import org.apache.flink.core.memory.DataInputView; +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +public class StateCheckpointReader { + + private final DataInputView input; + + public StateCheckpointReader(DataInputView in) { + this.input = in; + } + + public ByteString getTag() throws IOException { + return ByteString.copyFrom(readRawData()); + } + + public String getTagToString() throws IOException { + return input.readUTF(); + } + + public ByteString getData() throws IOException { + return ByteString.copyFrom(readRawData()); + } + + public int getInt() throws IOException { + validate(); + return input.readInt(); + } + + public byte getByte() throws IOException { + validate(); + return input.readByte(); + } + + public Instant getTimestamp() throws IOException { + validate(); + Long watermarkMillis = input.readLong(); + return new Instant(TimeUnit.MICROSECONDS.toMillis(watermarkMillis)); + } + + public K deserializeKey(CoderTypeSerializer keySerializer) throws IOException { + return deserializeObject(keySerializer); + } + + public T deserializeObject(CoderTypeSerializer objectSerializer) throws IOException { + return objectSerializer.deserialize(input); + } + + ///////// Helper Methods /////// + + private byte[] readRawData() throws IOException { + validate(); + int size = input.readInt(); + + byte[] serData = new byte[size]; + int bytesRead = input.read(serData); + if (bytesRead != size) { + throw new RuntimeException("Error while deserializing checkpoint. Not enough bytes in the input stream."); + } + return serData; + } + + private void validate() { + if (this.input == null) { + throw new RuntimeException("StateBackend not initialized yet."); + } + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java new file mode 100644 index 000000000000..6bc8662512f9 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java @@ -0,0 +1,152 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; + +import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.util.TimeDomain; +import com.google.cloud.dataflow.sdk.util.TimerInternals; +import com.google.cloud.dataflow.sdk.util.state.StateNamespace; +import com.google.cloud.dataflow.sdk.util.state.StateNamespaces; +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +public class StateCheckpointUtils { + + public static void encodeState(Map> perKeyStateInternals, + StateCheckpointWriter writer, Coder keyCoder) throws IOException { + CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); + + int noOfKeys = perKeyStateInternals.size(); + writer.writeInt(noOfKeys); + for (Map.Entry> keyStatePair : perKeyStateInternals.entrySet()) { + K key = keyStatePair.getKey(); + FlinkStateInternals state = keyStatePair.getValue(); + + // encode the key + writer.serializeKey(key, keySerializer); + + // write the associated state + state.persistState(writer); + } + } + + public static Map> decodeState( + StateCheckpointReader reader, + Combine.KeyedCombineFn combineFn, + Coder keyCoder, + Coder windowCoder, + ClassLoader classLoader) throws IOException, ClassNotFoundException { + + int noOfKeys = reader.getInt(); + Map> perKeyStateInternals = new HashMap<>(noOfKeys); + perKeyStateInternals.clear(); + + CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); + for (int i = 0; i < noOfKeys; i++) { + + // decode the key. + K key = reader.deserializeKey(keySerializer); + + //decode the state associated to the key. + FlinkStateInternals stateForKey = + new FlinkStateInternals<>(key, keyCoder, windowCoder, combineFn); + stateForKey.restoreState(reader, classLoader); + perKeyStateInternals.put(key, stateForKey); + } + return perKeyStateInternals; + } + + ////////////// Encoding/Decoding the Timers //////////////// + + + public static void encodeTimers(Map> allTimers, + StateCheckpointWriter writer, + Coder keyCoder) throws IOException { + CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); + + int noOfKeys = allTimers.size(); + writer.writeInt(noOfKeys); + for (Map.Entry> timersPerKey : allTimers.entrySet()) { + K key = timersPerKey.getKey(); + + // encode the key + writer.serializeKey(key, keySerializer); + + // write the associated timers + Set timers = timersPerKey.getValue(); + encodeTimerDataForKey(writer, timers); + } + } + + public static Map> decodeTimers( + StateCheckpointReader reader, + Coder windowCoder, + Coder keyCoder) throws IOException { + + int noOfKeys = reader.getInt(); + Map> activeTimers = new HashMap<>(noOfKeys); + activeTimers.clear(); + + CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); + for (int i = 0; i < noOfKeys; i++) { + + // decode the key. + K key = reader.deserializeKey(keySerializer); + + // decode the associated timers. + Set timers = decodeTimerDataForKey(reader, windowCoder); + activeTimers.put(key, timers); + } + return activeTimers; + } + + private static void encodeTimerDataForKey(StateCheckpointWriter writer, Set timers) throws IOException { + // encode timers + writer.writeInt(timers.size()); + for (TimerInternals.TimerData timer : timers) { + String stringKey = timer.getNamespace().stringKey(); + + writer.setTag(stringKey); + writer.setTimestamp(timer.getTimestamp()); + writer.writeInt(timer.getDomain().ordinal()); + } + } + + private static Set decodeTimerDataForKey( + StateCheckpointReader reader, Coder windowCoder) throws IOException { + + // decode the timers: first their number and then the content itself. + int noOfTimers = reader.getInt(); + Set timers = new HashSet<>(noOfTimers); + for (int i = 0; i < noOfTimers; i++) { + String stringKey = reader.getTagToString(); + Instant instant = reader.getTimestamp(); + TimeDomain domain = TimeDomain.values()[reader.getInt()]; + + StateNamespace namespace = StateNamespaces.fromString(stringKey, windowCoder); + timers.add(TimerInternals.TimerData.of(namespace, instant, domain)); + } + return timers; + } +} diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java new file mode 100644 index 000000000000..7201112c3c01 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java @@ -0,0 +1,127 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; + +import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer; +import com.google.protobuf.ByteString; +import org.apache.flink.runtime.state.StateBackend; +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +public class StateCheckpointWriter { + + private final StateBackend.CheckpointStateOutputView output; + + public static StateCheckpointWriter create(StateBackend.CheckpointStateOutputView output) { + return new StateCheckpointWriter(output); + } + + private StateCheckpointWriter(StateBackend.CheckpointStateOutputView output) { + this.output = output; + } + + ///////// Creating the serialized versions of the different types of state held by dataflow /////// + + public StateCheckpointWriter addValueBuilder() throws IOException { + validate(); + StateType.serialize(StateType.VALUE, this); + return this; + } + + public StateCheckpointWriter addWatermarkHoldsBuilder() throws IOException { + validate(); + StateType.serialize(StateType.WATERMARK, this); + return this; + } + + public StateCheckpointWriter addListUpdatesBuilder() throws IOException { + validate(); + StateType.serialize(StateType.LIST, this); + return this; + } + + public StateCheckpointWriter addAccumulatorBuilder() throws IOException { + validate(); + StateType.serialize(StateType.ACCUMULATOR, this); + return this; + } + + ///////// Setting the tag for a given state element /////// + + public StateCheckpointWriter setTag(ByteString stateKey) throws IOException { + return writeData(stateKey.toByteArray()); + } + + public StateCheckpointWriter setTag(String stateKey) throws IOException { + output.writeUTF(stateKey); + return this; + } + + + public StateCheckpointWriter serializeKey(K key, CoderTypeSerializer keySerializer) throws IOException { + return serializeObject(key, keySerializer); + } + + public StateCheckpointWriter serializeObject(T object, CoderTypeSerializer objectSerializer) throws IOException { + objectSerializer.serialize(object, output); + return this; + } + + ///////// Write the actual serialized data ////////// + + public StateCheckpointWriter setData(ByteString data) throws IOException { + return writeData(data.toByteArray()); + } + + public StateCheckpointWriter setData(byte[] data) throws IOException { + return writeData(data); + } + + public StateCheckpointWriter setTimestamp(Instant timestamp) throws IOException { + validate(); + output.writeLong(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis())); + return this; + } + + public StateCheckpointWriter writeInt(int number) throws IOException { + validate(); + output.writeInt(number); + return this; + } + + public StateCheckpointWriter writeByte(byte b) throws IOException { + validate(); + output.writeByte(b); + return this; + } + + ///////// Helper Methods /////// + + private StateCheckpointWriter writeData(byte[] data) throws IOException { + validate(); + output.writeInt(data.length); + output.write(data); + return this; + } + + private void validate() { + if (this.output == null) { + throw new RuntimeException("StateBackend not initialized yet."); + } + } +} \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java new file mode 100644 index 000000000000..11446ea48c3c --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java @@ -0,0 +1,67 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; + +import java.io.IOException; + +public enum StateType { + + VALUE(0), + + WATERMARK(1), + + LIST(2), + + ACCUMULATOR(3); + + private final int numVal; + + StateType(int value) { + this.numVal = value; + } + + public static void serialize(StateType type, StateCheckpointWriter output) throws IOException { + if (output == null) { + throw new IllegalArgumentException("Cannot write to a null output."); + } + + if(type.numVal < 0 || type.numVal > 3) { + throw new RuntimeException("Unknown State Type " + type + "."); + } + + output.writeByte((byte) type.numVal); + } + + public static StateType deserialize(StateCheckpointReader input) throws IOException { + if (input == null) { + throw new IllegalArgumentException("Cannot read from a null input."); + } + + int typeInt = (int) input.getByte(); + if(typeInt < 0 || typeInt > 3) { + throw new RuntimeException("Unknown State Type " + typeInt + "."); + } + + StateType resultType = null; + for(StateType st: values()) { + if(st.numVal == typeInt) { + resultType = st; + break; + } + } + return resultType; + } +} diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java new file mode 100644 index 000000000000..b6671876c547 --- /dev/null +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java @@ -0,0 +1,507 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.streaming; + +import com.dataartisans.flink.dataflow.FlinkTestPipeline; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.FlinkGroupAlsoByWindowWrapper; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.windowing.*; +import com.google.cloud.dataflow.sdk.util.UserCodeException; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import com.google.cloud.dataflow.sdk.util.WindowingStrategy; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.base.Throwables; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; + +import java.util.Collection; +import java.util.Comparator; +import java.util.concurrent.ConcurrentLinkedQueue; + +public class GroupAlsoByWindowTest { + + private final Combine.CombineFn combiner = new Sum.SumIntegerFn(); + + private final WindowingStrategy slidingWindowWithAfterWatermarkTriggerStrategy = + WindowingStrategy.of(SlidingWindows.of(Duration.standardSeconds(10)).every(Duration.standardSeconds(5))) + .withTrigger(AfterWatermark.pastEndOfWindow()).withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES); + + private final WindowingStrategy sessionWindowingStrategy = + WindowingStrategy.of(Sessions.withGapDuration(Duration.standardSeconds(2))) + .withTrigger(Repeatedly.forever(AfterWatermark.pastEndOfWindow())) + .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES) + .withAllowedLateness(Duration.standardSeconds(100)); + + private final WindowingStrategy fixedWindowingStrategy = + WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(10))); + + private final WindowingStrategy fixedWindowWithCountTriggerStrategy = + fixedWindowingStrategy.withTrigger(AfterPane.elementCountAtLeast(5)); + + private final WindowingStrategy fixedWindowWithAfterWatermarkTriggerStrategy = + fixedWindowingStrategy.withTrigger(AfterWatermark.pastEndOfWindow()); + + private final WindowingStrategy fixedWindowWithCompoundTriggerStrategy = + fixedWindowingStrategy.withTrigger( + Repeatedly.forever(AfterFirst.of( + AfterPane.elementCountAtLeast(5), + AfterWatermark.pastEndOfWindow()))); + + /** + * The default accumulation mode is + * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#DISCARDING_FIRED_PANES}. + * This strategy changes it to + * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#ACCUMULATING_FIRED_PANES} + */ + private final WindowingStrategy fixedWindowWithCompoundTriggerStrategyAcc = + fixedWindowWithCompoundTriggerStrategy + .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES); + + @Test + public void testWithLateness() throws Exception { + WindowingStrategy strategy = WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(2))) + .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES) + .withAllowedLateness(Duration.millis(1000)); + long initialTime = 0L; + Pipeline pipeline = FlinkTestPipeline.create(); + + KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); + + FlinkGroupAlsoByWindowWrapper gbwOperaror = + FlinkGroupAlsoByWindowWrapper.createForTesting( + pipeline.getOptions(), + pipeline.getCoderRegistry(), + strategy, + inputCoder, + combiner.asKeyedFn()); + + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + new OneInputStreamOperatorTestHarness<>(gbwOperaror); + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processWatermark(new Watermark(initialTime + 2000)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processWatermark(new Watermark(initialTime + 4000)); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 4), + new Instant(initialTime + 1), + new IntervalWindow(new Instant(0), new Instant(2000)), + PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) + , initialTime)); + expectedOutput.add(new Watermark(initialTime + 2000)); + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 6), + new Instant(initialTime + 1999), + new IntervalWindow(new Instant(0), new Instant(2000)), + PaneInfo.createPane(false, true, PaneInfo.Timing.LATE, 1, 1)) + , initialTime)); + + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 0), + new Instant(initialTime + 1999), + new IntervalWindow(new Instant(0), new Instant(2000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)) + , initialTime)); + expectedOutput.add(new Watermark(initialTime + 4000)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + testHarness.close(); + } + + @Test + public void testSessionWindows() throws Exception { + WindowingStrategy strategy = sessionWindowingStrategy; + + long initialTime = 0L; + Pipeline pipeline = FlinkTestPipeline.create(); + + KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); + + FlinkGroupAlsoByWindowWrapper gbwOperaror = + FlinkGroupAlsoByWindowWrapper.createForTesting( + pipeline.getOptions(), + pipeline.getCoderRegistry(), + strategy, + inputCoder, + combiner.asKeyedFn()); + + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + new OneInputStreamOperatorTestHarness<>(gbwOperaror); + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3700), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 2700), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processWatermark(new Watermark(initialTime + 6000)); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6700), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6800), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 8900), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 7600), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 5600), null, PaneInfo.NO_FIRING), initialTime + 20)); + + testHarness.processWatermark(new Watermark(initialTime + 12000)); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 6), + new Instant(initialTime + 1), + new IntervalWindow(new Instant(1), new Instant(5700)), + PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) + , initialTime)); + expectedOutput.add(new Watermark(initialTime + 6000)); + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 11), + new Instant(initialTime + 6700), + new IntervalWindow(new Instant(1), new Instant(10900)), + PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) + , initialTime)); + expectedOutput.add(new Watermark(initialTime + 12000)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + testHarness.close(); + } + + @Test + public void testSlidingWindows() throws Exception { + WindowingStrategy strategy = slidingWindowWithAfterWatermarkTriggerStrategy; + long initialTime = 0L; + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + createTestingOperatorAndState(strategy, initialTime); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + testHarness.processWatermark(new Watermark(initialTime + 25000)); + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 6), + new Instant(initialTime + 5000), + new IntervalWindow(new Instant(0), new Instant(10000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime)); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 6), + new Instant(initialTime + 1), + new IntervalWindow(new Instant(-5000), new Instant(5000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime)); + expectedOutput.add(new Watermark(initialTime + 10000)); + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 11), + new Instant(initialTime + 15000), + new IntervalWindow(new Instant(10000), new Instant(20000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime)); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 3), + new Instant(initialTime + 10000), + new IntervalWindow(new Instant(5000), new Instant(15000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime)); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key2", 1), + new Instant(initialTime + 19500), + new IntervalWindow(new Instant(10000), new Instant(20000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime)); + expectedOutput.add(new Watermark(initialTime + 20000)); + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key2", 1), + new Instant(initialTime + 20000), + /** + * this is 20000 and not 19500 because of a convention in dataflow where + * timestamps of windowed values in a window cannot be smaller than the + * end of a previous window. Checkout the documentation of the + * {@link WindowFn#getOutputTime(Instant, BoundedWindow)} + */ + new IntervalWindow(new Instant(15000), new Instant(25000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime)); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 8), + new Instant(initialTime + 20000), + new IntervalWindow(new Instant(15000), new Instant(25000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime)); + expectedOutput.add(new Watermark(initialTime + 25000)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + testHarness.close(); + } + + @Test + public void testAfterWatermarkProgram() throws Exception { + WindowingStrategy strategy = fixedWindowWithAfterWatermarkTriggerStrategy; + long initialTime = 0L; + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + createTestingOperatorAndState(strategy, initialTime); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6), + new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime)); + expectedOutput.add(new Watermark(initialTime + 10000)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11), + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime)); + expectedOutput.add(new Watermark(initialTime + 20000)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + testHarness.close(); + } + + @Test + public void testAfterCountProgram() throws Exception { + WindowingStrategy strategy = fixedWindowWithCountTriggerStrategy; + + long initialTime = 0L; + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + createTestingOperatorAndState(strategy, initialTime); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime)); + expectedOutput.add(new Watermark(initialTime + 10000)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.LATE, 0, 0)), initialTime)); + expectedOutput.add(new Watermark(initialTime + 20000)); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + + testHarness.close(); + } + + @Test + public void testCompoundProgram() throws Exception { + WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategy; + + long initialTime = 0L; + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + createTestingOperatorAndState(strategy, initialTime); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + /** + * PaneInfo are: + * isFirst (pane in window), + * isLast, Timing (of triggering), + * index (of pane in the window), + * onTimeIndex (if it the 1st,2nd, ... pane that was fired on time) + * */ + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), + new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime)); + + expectedOutput.add(new Watermark(initialTime + 10000)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime)); + + expectedOutput.add(new Watermark(initialTime + 20000)); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + + testHarness.close(); + } + + @Test + public void testCompoundAccumulatingPanesProgram() throws Exception { + WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc; + long initialTime = 0L; + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + createTestingOperatorAndState(strategy, initialTime); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 10), + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6), + new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime)); + + expectedOutput.add(new Watermark(initialTime + 10000)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11), + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime)); + + expectedOutput.add(new Watermark(initialTime + 20000)); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + + testHarness.close(); + } + + private OneInputStreamOperatorTestHarness createTestingOperatorAndState(WindowingStrategy strategy, long initialTime) throws Exception { + Pipeline pipeline = FlinkTestPipeline.create(); + + KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); + + FlinkGroupAlsoByWindowWrapper gbwOperaror = + FlinkGroupAlsoByWindowWrapper.createForTesting( + pipeline.getOptions(), + pipeline.getCoderRegistry(), + strategy, + inputCoder, + combiner.asKeyedFn()); + + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + new OneInputStreamOperatorTestHarness<>(gbwOperaror); + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 10000), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 12100), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 14200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 15300), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 16500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + + testHarness.processWatermark(new Watermark(initialTime + 10000)); + testHarness.processWatermark(new Watermark(initialTime + 20000)); + + return testHarness; + } + + private static class ResultSortComparator implements Comparator { + @Override + public int compare(Object o1, Object o2) { + if (o1 instanceof Watermark && o2 instanceof Watermark) { + Watermark w1 = (Watermark) o1; + Watermark w2 = (Watermark) o2; + return (int) (w1.getTimestamp() - w2.getTimestamp()); + } else { + StreamRecord>> sr0 = (StreamRecord>>) o1; + StreamRecord>> sr1 = (StreamRecord>>) o2; + + if (sr0.getTimestamp() != sr1.getTimestamp()) { + return (int) (sr0.getTimestamp() - sr1.getTimestamp()); + } + + int comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey()); + if(comparison == 0) { + comparison = Integer.compare( + sr0.getValue().getValue().getValue(), + sr1.getValue().getValue().getValue()); + } + if(comparison == 0) { + Collection windowsA = sr0.getValue().getWindows(); + Collection windowsB = sr1.getValue().getWindows(); + + if(windowsA.size() != 1 || windowsB.size() != 1) { + throw new IllegalStateException("A value cannot belong to more than one windows after grouping."); + } + + BoundedWindow windowA = (BoundedWindow) windowsA.iterator().next(); + BoundedWindow windowB = (BoundedWindow) windowsB.iterator().next(); + comparison = Long.compare(windowA.maxTimestamp().getMillis(), windowB.maxTimestamp().getMillis()); + } + return comparison; + } + } + } + + private WindowedValue makeWindowedValue(WindowingStrategy strategy, + T output, Instant timestamp, Collection windows, PaneInfo pane) { + final Instant inputTimestamp = timestamp; + final WindowFn windowFn = strategy.getWindowFn(); + + if (timestamp == null) { + timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + if (windows == null) { + try { + windows = windowFn.assignWindows(windowFn.new AssignContext() { + @Override + public Object element() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input element when none was available"); // TODO: 12/16/15 aljoscha's comment in slack + } + + @Override + public Instant timestamp() { + if (inputTimestamp == null) { + throw new UnsupportedOperationException( + "WindowFn attempted to access input timestamp when none was available"); + } + return inputTimestamp; + } + + @Override + public Collection windows() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input windows when none were available"); + } + }); + } catch (Exception e) { + Throwables.propagateIfInstanceOf(e, UserCodeException.class); + throw new UserCodeException(e); + } + } + + return WindowedValue.of(output, timestamp, windows, pane); + } +} diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java new file mode 100644 index 000000000000..084ada236570 --- /dev/null +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java @@ -0,0 +1,257 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.streaming; + +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.FlinkStateInternals; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointReader; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointUtils; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointWriter; +import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; +import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.Sum; +import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.util.TimeDomain; +import com.google.cloud.dataflow.sdk.util.TimerInternals; +import com.google.cloud.dataflow.sdk.util.state.*; +import org.apache.flink.api.java.typeutils.runtime.ByteArrayInputView; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.joda.time.Instant; +import org.junit.Test; + +import java.util.*; + +import static org.junit.Assert.assertEquals; + +public class StateSerializationTest { + + private static final StateNamespace NAMESPACE_1 = StateNamespaces.global(); + private static final String KEY_PREFIX = "TEST_"; + + private static final StateTag> STRING_VALUE_ADDR = + StateTags.value("stringValue", StringUtf8Coder.of()); + private static final StateTag> INT_VALUE_ADDR = + StateTags.value("stringValue", VarIntCoder.of()); + private static final StateTag> SUM_INTEGER_ADDR = + StateTags.combiningValueFromInputInternal( + "sumInteger", VarIntCoder.of(), new Sum.SumIntegerFn()); + private static final StateTag> STRING_BAG_ADDR = + StateTags.bag("stringBag", StringUtf8Coder.of()); + private static final StateTag WATERMARK_BAG_ADDR = + StateTags.watermarkStateInternal("watermark"); + + private Combine.CombineFn combiner = new Sum.SumIntegerFn(); + + private Map> statePerKey = new HashMap<>(); + + private Map> activeTimers = new HashMap<>(); + + private void initializeStateAndTimers() throws CannotProvideCoderException { + for (int i = 0; i < 10; i++) { + String key = KEY_PREFIX + i; + + FlinkStateInternals state = initializeStateForKey(key); + Set timers = new HashSet<>(); + for (int j = 0; j < 5; j++) { + TimerInternals.TimerData timer = TimerInternals + .TimerData.of(NAMESPACE_1, + new Instant(1000 + i + j), TimeDomain.values()[j % 3]); + timers.add(timer); + } + + statePerKey.put(key, state); + activeTimers.put(key, timers); + } + } + + private FlinkStateInternals initializeStateForKey(String key) throws CannotProvideCoderException { + FlinkStateInternals state = createState(key); + + ValueState value = state.state(NAMESPACE_1, STRING_VALUE_ADDR); + value.set("test"); + + ValueState value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR); + value2.set(4); + value2.set(5); + + CombiningValueState combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR); + combiningValue.add(1); + combiningValue.add(2); + + WatermarkStateInternal watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + watermark.add(new Instant(1000)); + + BagState bag = state.state(NAMESPACE_1, STRING_BAG_ADDR); + bag.add("v1"); + bag.add("v2"); + bag.add("v3"); + bag.add("v4"); + return state; + } + + private boolean restoreAndTestState(DataInputView in) throws Exception { + StateCheckpointReader reader = new StateCheckpointReader(in); + final ClassLoader userClassloader = this.getClass().getClassLoader(); + Coder windowCoder = IntervalWindow.getCoder(); + Coder keyCoder = StringUtf8Coder.of(); + + boolean comparisonRes = true; + + for(String key: statePerKey.keySet()) { + comparisonRes &= checkStateForKey(key); + } + + // restore the timers + Map> restoredTimersPerKey = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder); + if(activeTimers.size() != restoredTimersPerKey.size()) { + return false; + } + + for(String key: statePerKey.keySet()) { + Set originalTimers = activeTimers.get(key); + Set restoredTimers = restoredTimersPerKey.get(key); + comparisonRes &= checkTimersForKey(originalTimers, restoredTimers); + } + + // restore the state + Map> restoredPerKeyState = StateCheckpointUtils.decodeState(reader, combiner.asKeyedFn(), keyCoder, windowCoder, userClassloader); + if(restoredPerKeyState.size() != statePerKey.size()) { + return false; + } + + for(String key: statePerKey.keySet()) { + FlinkStateInternals originalState = statePerKey.get(key); + FlinkStateInternals restoredState = restoredPerKeyState.get(key); + comparisonRes &= checkStateForKey(originalState, restoredState); + } + return comparisonRes; + } + + private boolean checkStateForKey(String key) throws CannotProvideCoderException { + FlinkStateInternals state = statePerKey.get(key); + + ValueState value = state.state(NAMESPACE_1, STRING_VALUE_ADDR); + boolean comp = value.get().read().equals("test"); + + ValueState value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR); + comp &= value2.get().read().equals(5); + + CombiningValueState combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR); + comp &= combiningValue.get().read().equals(3); + + WatermarkStateInternal watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + comp &= watermark.get().read().equals(new Instant(1000)); + + BagState bag = state.state(NAMESPACE_1, STRING_BAG_ADDR); + Iterator it = bag.get().read().iterator(); + int i = 0; + while(it.hasNext()) { + comp &= it.next().equals("v"+ (++i)); + } + return comp; + } + + private void storeState(StateBackend.CheckpointStateOutputView out) throws Exception { + StateCheckpointWriter checkpointBuilder = StateCheckpointWriter.create(out); + Coder keyCoder = StringUtf8Coder.of(); + + // checkpoint the timers + StateCheckpointUtils.encodeTimers(activeTimers, checkpointBuilder,keyCoder); + + // checkpoint the state + StateCheckpointUtils.encodeState(statePerKey, checkpointBuilder, keyCoder); + } + + private boolean checkTimersForKey(Set originalTimers, Set restoredTimers) { + boolean comp = true; + if(restoredTimers == null) { + return false; + } + + if(originalTimers.size() != restoredTimers.size()) { + return false; + } + + for(TimerInternals.TimerData timer: originalTimers) { + comp &= restoredTimers.contains(timer); + } + return comp; + } + + private boolean checkStateForKey(FlinkStateInternals originalState, FlinkStateInternals restoredState) throws CannotProvideCoderException { + if(restoredState == null) { + return false; + } + + ValueState orValue = originalState.state(NAMESPACE_1, STRING_VALUE_ADDR); + ValueState resValue = restoredState.state(NAMESPACE_1, STRING_VALUE_ADDR); + boolean comp = orValue.get().read().equals(resValue.get().read()); + + ValueState orIntValue = originalState.state(NAMESPACE_1, INT_VALUE_ADDR); + ValueState resIntValue = restoredState.state(NAMESPACE_1, INT_VALUE_ADDR); + comp &= orIntValue.get().read().equals(resIntValue.get().read()); + + CombiningValueState combOrValue = originalState.state(NAMESPACE_1, SUM_INTEGER_ADDR); + CombiningValueState combResValue = restoredState.state(NAMESPACE_1, SUM_INTEGER_ADDR); + comp &= combOrValue.get().read().equals(combResValue.get().read()); + + WatermarkStateInternal orWatermark = originalState.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + WatermarkStateInternal resWatermark = restoredState.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + comp &= orWatermark.get().read().equals(resWatermark.get().read()); + + BagState orBag = originalState.state(NAMESPACE_1, STRING_BAG_ADDR); + BagState resBag = restoredState.state(NAMESPACE_1, STRING_BAG_ADDR); + + Iterator orIt = orBag.get().read().iterator(); + Iterator resIt = resBag.get().read().iterator(); + + while (orIt.hasNext() && resIt.hasNext()) { + comp &= orIt.next().equals(resIt.next()); + } + + return !((orIt.hasNext() && !resIt.hasNext()) || (!orIt.hasNext() && resIt.hasNext())) && comp; + } + + private FlinkStateInternals createState(String key) throws CannotProvideCoderException { + return new FlinkStateInternals<>( + key, + StringUtf8Coder.of(), + IntervalWindow.getCoder(), + combiner.asKeyedFn()); + } + + @Test + public void test() throws Exception { + StateSerializationTest test = new StateSerializationTest(); + test.initializeStateAndTimers(); + + MemoryStateBackend.MemoryCheckpointOutputStream memBackend = new MemoryStateBackend.MemoryCheckpointOutputStream(25728); + StateBackend.CheckpointStateOutputView out = new StateBackend.CheckpointStateOutputView(memBackend); + + test.storeState(out); + + byte[] contents = memBackend.closeAndGetBytes(); + ByteArrayInputView in = new ByteArrayInputView(contents); + + assertEquals(test.restoreAndTestState(in), true); + } + +} diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java index cbf5d77744b6..74f754b271db 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java @@ -76,8 +76,8 @@ public void processElement(ProcessContext c) { KV e = c.element(); CoGbkResult val = e.getValue(); String countryCode = e.getKey(); - String countryName; - countryName = e.getValue().getOnly(countryInfoTag); + String countryName = "none"; + countryName = e.getValue().getOnly(countryInfoTag, "Kostas"); for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) { // Generate a string that combines information from both collection values c.output(KV.of(countryCode, "Country name: " + countryName From 9bfdea22b5729f8d49ee9a3ad4ba9364f93cb98b Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 20 Jan 2016 17:25:13 +0100 Subject: [PATCH 126/149] [readme] add hint that streaming support is available --- runners/flink/README.md | 8 ++++++-- .../wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java | 2 +- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/runners/flink/README.md b/runners/flink/README.md index 8c06c1da3414..54d248cf7917 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -142,5 +142,9 @@ the [Mailinglists](http://flink.apache.org/community.html#mailing-lists). # Streaming -Streaming support is currently under development. See the `streaming_new` branch for the current -work in progress version. +Streaming support has been added. It is currently in alpha stage. Please give it a try. To use +streaming, just enable streaming mode in the `PipelineOptions`: + + options.setStreaming(true); + +That's all. \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java index c52fabea533f..0f0a9d0daace 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -48,7 +48,7 @@ /** * This class is the key class implementing all the windowing/triggering logic of Google Dataflow. * To provide full compatibility and support all the windowing/triggering combinations offered by - * Datadlow, we opted for a strategy that uses the SDK's code for doing these operations + * Dataflow, we opted for a strategy that uses the SDK's code for doing these operations * ({@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn}. *

    * In a nutshell, when the execution arrives to this operator, we expect to have a stream already From 8057fc25c7a30431b7896e98f96d4f07bde389f4 Mon Sep 17 00:00:00 2001 From: Max Date: Thu, 11 Feb 2016 12:36:02 +0100 Subject: [PATCH 127/149] [readme] update to reflect the current state --- runners/flink/README.md | 82 +++++++++++++++++++++++++++++++++-------- 1 file changed, 67 insertions(+), 15 deletions(-) diff --git a/runners/flink/README.md b/runners/flink/README.md index 54d248cf7917..499ed6d0b72a 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -1,13 +1,72 @@ Flink-Dataflow -------------- -Flink-Dataflow is a Google Dataflow Runner for Apache Flink. It enables you to -run Dataflow programs with Flink as an execution engine. +Flink-Dataflow is a Runner for Google Dataflow (aka Apache Beam) which enables you to +run Dataflow programs with Flink. It integrates seamlessly with the Dataflow +API, allowing you to execute Dataflow programs in streaming or batch mode. + +## Streaming + +### Full Dataflow Windowing and Triggering Semantics + +The Flink Dataflow Runner supports *Event Time* allowing you to analyze data with respect to its +associated timestamp. It handles out-or-order and late-arriving elements. You may leverage the full +power of the Dataflow windowing semantics like *time-based*, *sliding*, *tumbling*, or *count* +windows. You may build *session* windows which allow you to keep track of events associated with +each other. + +### Fault-Tolerance + +The program's state is persisted by Apache Flink. You may re-run and resume your program upon +failure or if you decide to continue computation at a later time. + +### Sources and Sinks + +Build your own data ingestion or digestion using the source/sink interface. Re-use Flink's sources +and sinks or use the provided support for Apache Kafka. + +### Seamless integration + +To execute a Dataflow program in streaming mode, just enable streaming in the `PipelineOptions`: + + options.setStreaming(true); + +That's it. If you prefer batched execution, simply disable streaming mode. + +## Batch + +### Batch optimization + +Flink gives you out-of-core algorithms which operate on its managed memory to perform sorting, +caching, and hash table operations. We have optimized operations like CoGroup to use Flink's +optimized out-of-core implementation. + +### Fault-Tolerance + +We guarantee job-level fault-tolerance which gracefully restarts failed batch jobs. + +### Sources and Sinks + +Build your own data ingestion or digestion using the source/sink interface or re-use Flink's sources +and sinks. + +## Features + +The Flink Dataflow Runner maintains as much compatibility with the Dataflow API as possible. We +support transformations on data like: + +- Grouping +- Windowing +- ParDo +- CoGroup +- Flatten +- Combine +- Side inputs/outputs +- Encoding # Getting Started -To get started using Google Dataflow on top of Apache Flink, we need to install the -latest version of Flink-Dataflow. +To get started using Flink-Dataflow, we first need to install the latest version. ## Install Flink-Dataflow ## @@ -46,7 +105,6 @@ p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) p.run(); ``` - To execute the example, let's first get some sample data: curl http://www.gutenberg.org/cache/epub/1128/pg1128.txt > kinglear.txt @@ -58,7 +116,7 @@ Then let's run the included WordCount locally on your machine: Congratulations, you have run your first Google Dataflow program on top of Apache Flink! -# Running Dataflow on Flink on a cluster +# Running Dataflow programs on a Flink cluster You can run your Dataflow program on an Apache Flink cluster. Please start off by creating a new Maven project. @@ -137,14 +195,8 @@ folder to the Flink cluster using the command-line utility like so: ./bin/flink run /path/to/fat.jar -For more information, please visit the [Apache Flink Website](http://flink.apache.org) or contact -the [Mailinglists](http://flink.apache.org/community.html#mailing-lists). - -# Streaming -Streaming support has been added. It is currently in alpha stage. Please give it a try. To use -streaming, just enable streaming mode in the `PipelineOptions`: +# More - options.setStreaming(true); - -That's all. \ No newline at end of file +For more information, please visit the [Apache Flink Website](http://flink.apache.org) or contact +the [Mailinglists](http://flink.apache.org/community.html#mailing-lists). \ No newline at end of file From 9e98022cf43b24aacb92ca338736e807f8e4445c Mon Sep 17 00:00:00 2001 From: smarthi Date: Thu, 11 Feb 2016 17:33:51 -0500 Subject: [PATCH 128/149] [cleanup] various small improvements - removed duplicate declarations in pom.xml - removed reference to junit.framework.* - removed 'static' from interface declarations --- runners/flink/pom.xml | 5 ----- .../flink/dataflow/FlinkJobExecutionEnvironment.java | 3 +-- .../flink/dataflow/examples/streaming/AutoComplete.java | 6 ++---- .../flink/dataflow/examples/streaming/JoinExamples.java | 6 +++--- .../examples/streaming/KafkaWindowedWordCountExample.java | 2 +- .../dataflow/examples/streaming/WindowedWordCount.java | 3 +-- .../dataflow/translation/FlinkBatchPipelineTranslator.java | 3 --- .../translation/FlinkBatchTransformTranslators.java | 2 +- .../translation/FlinkStreamingPipelineTranslator.java | 4 ---- .../translation/FlinkStreamingTransformTranslators.java | 5 ++--- .../translation/FlinkStreamingTranslationContext.java | 6 ++---- .../functions/FlinkCoGroupKeyedListAggregator.java | 2 +- .../dataflow/translation/wrappers/SinkOutputFormat.java | 2 -- .../dataflow/translation/wrappers/SourceInputFormat.java | 7 +++---- .../com/dataartisans/flink/dataflow/WriteSinkITCase.java | 2 +- .../com/dataartisans/flink/dataflow/util/JoinExamples.java | 2 +- 16 files changed, 19 insertions(+), 41 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 14693b8d0322..cb784a080bc0 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -69,11 +69,6 @@ flink-core ${flink.version} - - org.apache.flink - flink-streaming-java - ${flink.version} - org.apache.flink flink-streaming-java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java index 66d60fa5a9b6..91b2f64d4bcb 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java @@ -23,7 +23,6 @@ import org.apache.flink.api.java.CollectionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.environment.RemoteStreamEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -224,7 +223,7 @@ private void createStreamExecutionEnvironment() { this.flinkStreamEnv.getConfig().setExecutionRetryDelay(3000); } - private final void checkInitializationState() { + private void checkInitializationState() { if (this.options == null) { throw new IllegalStateException("FlinkJobExecutionEnvironment is not initialized yet."); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java index 0245a7b21d15..711d9fbc1612 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java @@ -36,8 +36,6 @@ import java.io.IOException; import java.util.List; -import java.util.regex.Matcher; -import java.util.regex.Pattern; /** * To run the example, first open a socket on a terminal by executing the command: @@ -242,7 +240,7 @@ public AllPrefixes(int minPrefix, int maxPrefix) { public void processElement(ProcessContext c) { String word = c.element().value; for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) { - KV kv = KV.of(word.substring(0, i), c.element()); + KV kv = KV.of(word.substring(0, i), c.element()); c.output(kv); } } @@ -349,7 +347,7 @@ public void processElement(ProcessContext c) { * *

    Inherits standard Dataflow configuration options. */ - private static interface Options extends WindowedWordCount.StreamingWordCountOptions { + private interface Options extends WindowedWordCount.StreamingWordCountOptions { @Description("Whether to use the recursive algorithm") @Default.Boolean(true) Boolean getRecursive(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java index b0cc4fa1fc42..9a5db6463ae9 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java @@ -53,8 +53,8 @@ public class JoinExamples { static PCollection joinEvents(PCollection streamA, PCollection streamB) throws Exception { - final TupleTag firstInfoTag = new TupleTag(); - final TupleTag secondInfoTag = new TupleTag(); + final TupleTag firstInfoTag = new TupleTag<>(); + final TupleTag secondInfoTag = new TupleTag<>(); // transform both input collections to tuple collections, where the keys are country // codes in both cases. @@ -118,7 +118,7 @@ public void processElement(ProcessContext c) { } } - private static interface Options extends WindowedWordCount.StreamingWordCountOptions { + private interface Options extends WindowedWordCount.StreamingWordCountOptions { } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java index 46c9bd602462..42d3d883b2c0 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java @@ -72,7 +72,7 @@ public void processElement(ProcessContext c) { } } - public static interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions { + public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions { @Description("The Kafka topic to read from") @Default.String(KAFKA_TOPIC) String getKafkaTopic(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java index 1d4a44b494fd..b539245cf3f3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java @@ -28,7 +28,6 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import org.joda.time.Duration; -import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -81,7 +80,7 @@ public void processElement(ProcessContext c) { } } - public static interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options { + public interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options { @Description("Sliding window duration, in seconds") @Default.Long(WINDOW_SIZE) Long getWindowSize(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java index 8c0183ea6a1a..a1e441089117 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java @@ -115,9 +115,6 @@ public void visitValue(PValue value, TransformTreeNode producer) { } private > void applyBatchTransform(PTransform transform, TransformTreeNode node, BatchTransformTranslator translator) { - if (this.batchContext == null) { - throw new IllegalStateException("The FlinkPipelineTranslator is not yet initialized."); - } @SuppressWarnings("unchecked") T typedTransform = (T) transform; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java index 8f647300d2da..9a43d0507f52 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java @@ -283,7 +283,7 @@ public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslation private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator { @Override public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) { - PValue input = (PValue) context.getInput(transform); + PValue input = context.getInput(transform); DataSet inputDataSet = context.getInputDataSet(input); inputDataSet.printOnTaskManager(transform.getName()); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java index c8760c7ecbe1..a8f42263e6aa 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java @@ -19,7 +19,6 @@ import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey; import com.google.cloud.dataflow.sdk.values.PValue; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -101,9 +100,6 @@ public void visitValue(PValue value, TransformTreeNode producer) { } private > void applyStreamingTransform(PTransform transform, TransformTreeNode node, StreamTransformTranslator translator) { - if (this.streamingContext == null) { - throw new IllegalStateException("The FlinkPipelineTranslator is not yet initialized."); - } @SuppressWarnings("unchecked") T typedTransform = (T) transform; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java index 4c8cd4bf496f..17583cdf597f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java @@ -76,8 +76,7 @@ public class FlinkStreamingTransformTranslators { } public static FlinkStreamingPipelineTranslator.StreamTransformTranslator getTranslator(PTransform transform) { - FlinkStreamingPipelineTranslator.StreamTransformTranslator translator = TRANSLATORS.get(transform.getClass()); - return translator; + return TRANSLATORS.get(transform.getClass()); } // -------------------------------------------------------------------------------------------- @@ -123,7 +122,7 @@ private static class UnboundedReadSourceTranslator implements FlinkStreamingP public void translateNode(Read.Unbounded transform, FlinkStreamingTranslationContext context) { PCollection output = context.getOutput(transform); - DataStream> source = null; + DataStream> source; if (transform.getSource().getClass().equals(UnboundedFlinkSource.class)) { UnboundedFlinkSource flinkSource = (UnboundedFlinkSource) transform.getSource(); source = context.getExecutionEnvironment() diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java index 83ea5753abde..df68e5072afd 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java @@ -74,13 +74,11 @@ public void setCurrentTransform(AppliedPTransform currentTransform) { @SuppressWarnings("unchecked") public I getInput(PTransform transform) { - I input = (I) currentTransform.getInput(); - return input; + return (I) currentTransform.getInput(); } @SuppressWarnings("unchecked") public O getOutput(PTransform transform) { - O output = (O) currentTransform.getOutput(); - return output; + return (O) currentTransform.getOutput(); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java index f8593488100f..4c7fefd6b9b6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java @@ -53,6 +53,6 @@ public void coGroup(Iterable> first, Iterable> second, Collect k = entry.getKey(); result.add(new RawUnionValue(index2, entry.getValue())); } - out.collect(KV.of(k, new CoGbkResult(schema, (List) result))); + out.collect(KV.of(k, new CoGbkResult(schema, result))); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java index b10c86f9c907..ec8c186c5583 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java @@ -23,7 +23,6 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; import com.google.cloud.dataflow.sdk.transforms.Write; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.AbstractID; @@ -32,7 +31,6 @@ import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.lang.reflect.Field; -import java.util.UUID; /** * Wrapper class to use generic Write.Bound transforms as sinks. diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java index afb15da51fa0..b3eca96765e3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java @@ -45,7 +45,6 @@ public class SourceInputFormat implements InputFormat> private final BoundedSource initialSource; private transient PipelineOptions options; - private final Coder coder; private BoundedSource.BoundedReader reader = null; private boolean reachedEnd = true; @@ -53,7 +52,7 @@ public class SourceInputFormat implements InputFormat> public SourceInputFormat(BoundedSource initialSource, PipelineOptions options, Coder coder) { this.initialSource = initialSource; this.options = options; - this.coder = coder; + Coder coder1 = coder; } private void writeObject(ObjectOutputStream out) @@ -111,12 +110,12 @@ public float getAverageRecordWidth() { @Override @SuppressWarnings("unchecked") public SourceInputSplit[] createInputSplits(int numSplits) throws IOException { - long desiredSizeBytes = 10000; + long desiredSizeBytes; try { desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits; List> shards = initialSource.splitIntoBundles(desiredSizeBytes, options); - List> splits = new ArrayList>(); + List> splits = new ArrayList<>(); int splitCount = 0; for (Source shard: shards) { splits.add(new SourceInputSplit<>(shard, splitCount++)); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java index c8302e89c630..205fe9bddc2d 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java @@ -32,7 +32,7 @@ import java.io.PrintWriter; import java.net.URI; -import static junit.framework.Assert.*; +import static org.junit.Assert.*; /** * Tests the translation of custom Write.Bound sinks. diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java index 74f754b271db..aa5623d6c39d 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java @@ -76,7 +76,7 @@ public void processElement(ProcessContext c) { KV e = c.element(); CoGbkResult val = e.getValue(); String countryCode = e.getKey(); - String countryName = "none"; + String countryName; countryName = e.getValue().getOnly(countryInfoTag, "Kostas"); for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) { // Generate a string that combines information from both collection values From b1680104e0cf6b2f5be975294b886107335786ae Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 17 Feb 2016 13:23:26 +0100 Subject: [PATCH 129/149] [cleanup] remove obsolete code --- .../FlinkBatchTransformTranslators.java | 3 ++- .../translation/wrappers/SourceInputFormat.java | 3 +-- .../streaming/FlinkAbstractParDoWrapper.java | 14 -------------- .../flink/dataflow/JoinExamplesITCase.java | 7 ------- 4 files changed, 3 insertions(+), 24 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java index 9a43d0507f52..d5c09b2be10c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java @@ -151,7 +151,8 @@ public void translateNode(Read.Bounded transform, FlinkBatchTranslationContex TypeInformation typeInformation = context.getTypeInfo(output); - DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), new SourceInputFormat<>(source, context.getPipelineOptions(), coder), typeInformation, name); + DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), + new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name); context.setOutputDataSet(output, dataSource); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java index b3eca96765e3..64dc07267e7c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java @@ -49,10 +49,9 @@ public class SourceInputFormat implements InputFormat> private BoundedSource.BoundedReader reader = null; private boolean reachedEnd = true; - public SourceInputFormat(BoundedSource initialSource, PipelineOptions options, Coder coder) { + public SourceInputFormat(BoundedSource initialSource, PipelineOptions options) { this.initialSource = initialSource; this.options = options; - Coder coder1 = coder; } private void writeObject(ObjectOutputStream out) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index 53bb1778c227..71f9c7f089a2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -55,20 +55,6 @@ public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy function, Collector> outCollector) { if (this.context == null) { this.context = new DoFnProcessContext(function, outCollector); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java index dfcadc19a861..ed2ecf546406 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java @@ -53,13 +53,6 @@ public JoinExamplesITCase(){ }; static final List EVENT_ARRAY = Arrays.asList(EVENTS); - private static final KV kv1 = KV.of("VM", - "Date: 20141212, Actor1: LAOS, url: http://www.chicagotribune.com"); - private static final KV kv2 = KV.of("BE", - "Date: 20141213, Actor1: AFGHANISTAN, url: http://cnn.com"); - private static final KV kv3 = KV.of("BE", "Belgium"); - private static final KV kv4 = KV.of("VM", "Vietnam"); - private static final TableRow cc1 = new TableRow() .set("FIPSCC", "VM").set("HumanName", "Vietnam"); private static final TableRow cc2 = new TableRow() From 0628bf6a90de27aea8ad9ed48b61f98aa06a89f4 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 22 Feb 2016 16:31:23 +0100 Subject: [PATCH 130/149] [tests] add streaming mode to TestPipeline --- .../flink/dataflow/FlinkPipelineRunner.java | 3 +- .../flink/dataflow/FlinkTestPipeline.java | 33 +++++++++++++++++-- 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java index f57fed2dae20..ebd2691a7529 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java @@ -150,11 +150,12 @@ public FlinkPipelineOptions getPipelineOptions() { * * @return The newly created runner. */ - public static FlinkPipelineRunner createForTest() { + public static FlinkPipelineRunner createForTest(boolean streaming) { FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class); // we use [auto] for testing since this will make it pick up the Testing // ExecutionEnvironment options.setFlinkMaster("[auto]"); + options.setStreaming(streaming); return new FlinkPipelineRunner(options); } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java index a1f66c79f205..109b1ff8e75a 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java @@ -27,14 +27,41 @@ public class FlinkTestPipeline extends Pipeline { /** - * Creates and returns a new test pipeline. + * Creates and returns a new test pipeline for batch execution. * *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call * {@link Pipeline#run} to execute the pipeline and check the tests. */ public static FlinkTestPipeline create() { - FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(); - return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions()); + return create(false); + } + + /** + * Creates and returns a new test pipeline for streaming execution. + * + *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call + * {@link Pipeline#run} to execute the pipeline and check the tests. + * + * @return The Test Pipeline + */ + public static FlinkTestPipeline createStreaming() { + return create(true); + } + + /** + * Creates and returns a new test pipeline for streaming or batch execution. + * + *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call + * {@link Pipeline#run} to execute the pipeline and check the tests. + * + * @param streaming True for streaming mode, False for batch + * @return The Test Pipeline + */ + public static FlinkTestPipeline create(boolean streaming) { + FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming); + FlinkPipelineOptions pipelineOptions = flinkRunner.getPipelineOptions(); + pipelineOptions.setStreaming(streaming); + return new FlinkTestPipeline(flinkRunner, pipelineOptions); } private FlinkTestPipeline(PipelineRunner runner, PipelineOptions From 620e13bb2f5266bed0ad902be61ba3c9676ed6d0 Mon Sep 17 00:00:00 2001 From: Max Date: Mon, 22 Feb 2016 18:25:23 +0100 Subject: [PATCH 131/149] [runner] add Create transform --- .../FlinkStreamingTransformTranslators.java | 60 ++++++++++++++++-- .../io/FlinkStreamingCreateFunction.java | 61 +++++++++++++++++++ 2 files changed, 117 insertions(+), 4 deletions(-) create mode 100644 runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java index 17583cdf597f..46d3e36551df 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java @@ -13,11 +13,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.dataartisans.flink.dataflow.translation; import com.dataartisans.flink.dataflow.translation.functions.UnionCoder; import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; import com.dataartisans.flink.dataflow.translation.wrappers.streaming.*; +import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.FlinkStreamingCreateFunction; import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedFlinkSource; import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSourceWrapper; import com.google.api.client.util.Maps; @@ -37,6 +39,7 @@ import com.google.common.collect.Lists; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.streaming.api.datastream.*; import org.apache.flink.util.Collector; @@ -44,6 +47,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.util.*; /** @@ -64,6 +69,8 @@ public class FlinkStreamingTransformTranslators { // here you can find all the available translators. static { + + TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator()); TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator()); TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator()); TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator()); @@ -83,6 +90,47 @@ public static FlinkStreamingPipelineTranslator.StreamTransformTranslator getT // Transformation Implementations // -------------------------------------------------------------------------------------------- + private static class CreateStreamingTranslator implements + FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(Create.Values transform, FlinkStreamingTranslationContext context) { + PCollection output = context.getOutput(transform); + Iterable elements = transform.getElements(); + + // we need to serialize the elements to byte arrays, since they might contain + // elements that are not serializable by Java serialization. We deserialize them + // in the FlatMap function using the Coder. + + List serializedElements = Lists.newArrayList(); + Coder elementCoder = context.getOutput(transform).getCoder(); + for (OUT element: elements) { + ByteArrayOutputStream bao = new ByteArrayOutputStream(); + try { + elementCoder.encode(element, bao, Coder.Context.OUTER); + serializedElements.add(bao.toByteArray()); + } catch (IOException e) { + throw new RuntimeException("Could not serialize Create elements using Coder: " + e); + } + } + + + DataStream initDataSet = context.getExecutionEnvironment().fromElements(1); + + FlinkStreamingCreateFunction createFunction = + new FlinkStreamingCreateFunction<>(serializedElements, elementCoder); + + WindowedValue.ValueOnlyWindowedValueCoder windowCoder = WindowedValue.getValueOnlyCoder(elementCoder); + TypeInformation> outputType = new CoderTypeInformation<>(windowCoder); + + DataStream> outputDataStream = initDataSet.flatMap(createFunction) + .returns(outputType); + + context.setOutputDataStream(context.getOutput(transform), outputDataStream); + } + } + + private static class TextIOWriteBoundStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class); @@ -151,12 +199,16 @@ public void translateNode(ParDo.Bound transform, FlinkStreamingTranslat (WindowingStrategy) context.getOutput(transform).getWindowingStrategy(); - WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(), windowingStrategy.getWindowFn().windowCoder()); - CoderTypeInformation> outputWindowedValueCoder = new CoderTypeInformation<>(outputStreamCoder); + WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(), + windowingStrategy.getWindowFn().windowCoder()); + CoderTypeInformation> outputWindowedValueCoder = + new CoderTypeInformation<>(outputStreamCoder); - FlinkParDoBoundWrapper doFnWrapper = new FlinkParDoBoundWrapper<>(context.getPipelineOptions(), windowingStrategy, transform.getFn()); + FlinkParDoBoundWrapper doFnWrapper = new FlinkParDoBoundWrapper<>( + context.getPipelineOptions(), windowingStrategy, transform.getFn()); DataStream> inputDataStream = context.getInputDataStream(context.getInput(transform)); - SingleOutputStreamOperator, ?> outDataStream = inputDataStream.flatMap(doFnWrapper).returns(outputWindowedValueCoder); + SingleOutputStreamOperator, ?> outDataStream = inputDataStream.flatMap(doFnWrapper) + .returns(outputWindowedValueCoder); context.setOutputDataStream(context.getOutput(transform), outDataStream); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java new file mode 100644 index 000000000000..b8824f5763f7 --- /dev/null +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java @@ -0,0 +1,61 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.io; + +import com.dataartisans.flink.dataflow.translation.types.VoidCoderTypeSerializer; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; +import com.google.cloud.dataflow.sdk.util.WindowedValue; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.util.Collector; + +import java.io.ByteArrayInputStream; +import java.util.List; + +/** + * This flat map function bootstraps from collection elements and turns them into WindowedValues + * (as required by the Flink runner). + */ +public class FlinkStreamingCreateFunction implements FlatMapFunction> { + + private final List elements; + private final Coder coder; + + public FlinkStreamingCreateFunction(List elements, Coder coder) { + this.elements = elements; + this.coder = coder; + } + + @Override + public void flatMap(IN value, Collector> out) throws Exception { + + @SuppressWarnings("unchecked") + // TODO Flink doesn't allow null values in records + OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE; + + for (byte[] element : elements) { + ByteArrayInputStream bai = new ByteArrayInputStream(element); + OUT outValue = coder.decode(bai, Coder.Context.OUTER); + + if (outValue == null) { + out.collect(WindowedValue.of(voidValue, GlobalWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + } else { + out.collect(WindowedValue.of(outValue, GlobalWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + } + } + } +} From 7283d7a0e34c80fc4ef039eb4fcb71933d9585c6 Mon Sep 17 00:00:00 2001 From: Max Date: Tue, 23 Feb 2016 08:30:34 +0100 Subject: [PATCH 132/149] [tests] integrate Wikipedia session test --- .../flink/dataflow/TopWikipediaSessions.java | 210 ----------------- .../dataflow/TopWikipediaSessionsITCase.java | 215 ++++++++++++------ 2 files changed, 144 insertions(+), 281 deletions(-) delete mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java deleted file mode 100644 index ab5565a7e220..000000000000 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java +++ /dev/null @@ -1,210 +0,0 @@ -///* -// * Copyright (C) 2015 Google Inc. -// * -// * Licensed 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 com.dataartisans.flink.dataflow; -// -//import com.google.api.services.bigquery.model.TableRow; -//import com.google.cloud.dataflow.sdk.Pipeline; -//import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder; -//import com.google.cloud.dataflow.sdk.io.TextIO; -//import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; -//import com.google.cloud.dataflow.sdk.options.Default; -//import com.google.cloud.dataflow.sdk.options.Description; -//import com.google.cloud.dataflow.sdk.options.PipelineOptions; -//import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -//import com.google.cloud.dataflow.sdk.options.Validation; -//import com.google.cloud.dataflow.sdk.transforms.Count; -//import com.google.cloud.dataflow.sdk.transforms.DoFn; -//import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess; -//import com.google.cloud.dataflow.sdk.transforms.PTransform; -//import com.google.cloud.dataflow.sdk.transforms.ParDo; -//import com.google.cloud.dataflow.sdk.transforms.SerializableComparator; -//import com.google.cloud.dataflow.sdk.transforms.Top; -//import com.google.cloud.dataflow.sdk.transforms.windowing.CalendarWindows; -//import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; -//import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; -//import com.google.cloud.dataflow.sdk.transforms.windowing.Window; -//import com.google.cloud.dataflow.sdk.values.KV; -//import com.google.cloud.dataflow.sdk.values.PCollection; -// -//import org.joda.time.Duration; -//import org.joda.time.Instant; -// -//import java.util.List; -// -///** -// * Copied from {@link com.google.cloud.dataflow.examples.complete.TopWikipediaSessions} because the code -// * is private there. -// */ -//public class TopWikipediaSessions { -// private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json"; -// -// /** -// * Extracts user and timestamp from a TableRow representing a Wikipedia edit. -// */ -// static class ExtractUserAndTimestamp extends DoFn { -// private static final long serialVersionUID = 0; -// -// @Override -// public void processElement(ProcessContext c) { -// TableRow row = c.element(); -// int timestamp = (Integer) row.get("timestamp"); -// String userName = (String) row.get("contributor_username"); -// if (userName != null) { -// // Sets the implicit timestamp field to be used in windowing. -// c.outputWithTimestamp(userName, new Instant(timestamp * 1000L)); -// } -// } -// } -// -// /** -// * Computes the number of edits in each user session. A session is defined as -// * a string of edits where each is separated from the next by less than an hour. -// */ -// static class ComputeSessions -// extends PTransform, PCollection>> { -// private static final long serialVersionUID = 0; -// -// @Override -// public PCollection> apply(PCollection actions) { -// return actions -// .apply(Window.into(Sessions.withGapDuration(Duration.standardHours(1)))) -// -// .apply(Count.perElement()); -// } -// } -// -// /** -// * Computes the longest session ending in each month. -// */ -// private static class TopPerMonth -// extends PTransform>, PCollection>>> { -// private static final long serialVersionUID = 0; -// -// @Override -// public PCollection>> apply(PCollection> sessions) { -// return sessions -// .apply(Window.>into(CalendarWindows.months(1))) -// -// .apply(Top.of(1, new SerializableComparator>() { -// private static final long serialVersionUID = 0; -// -// @Override -// public int compare(KV o1, KV o2) { -// return Long.compare(o1.getValue(), o2.getValue()); -// } -// }).withoutDefaults()); -// } -// } -// -// static class SessionsToStringsDoFn extends DoFn, KV> -// implements RequiresWindowAccess { -// -// private static final long serialVersionUID = 0; -// -// @Override -// public void processElement(ProcessContext c) { -// c.output(KV.of( -// c.element().getKey() + " : " + c.window(), c.element().getValue())); -// } -// } -// -// static class FormatOutputDoFn extends DoFn>, String> -// implements RequiresWindowAccess { -// private static final long serialVersionUID = 0; -// -// @Override -// public void processElement(ProcessContext c) { -// for (KV item : c.element()) { -// String session = item.getKey(); -// long count = item.getValue(); -// c.output(session + " : " + count + " : " + ((IntervalWindow) c.window()).start()); -// } -// } -// } -// -// static class ComputeTopSessions extends PTransform, PCollection> { -// -// private static final long serialVersionUID = 0; -// -// private final double samplingThreshold; -// -// public ComputeTopSessions(double samplingThreshold) { -// this.samplingThreshold = samplingThreshold; -// } -// -// @Override -// public PCollection apply(PCollection input) { -// return input -// .apply(ParDo.of(new ExtractUserAndTimestamp())) -// -// .apply(ParDo.named("SampleUsers").of( -// new DoFn() { -// private static final long serialVersionUID = 0; -// -// @Override -// public void processElement(ProcessContext c) { -// if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * samplingThreshold) { -// c.output(c.element()); -// } -// } -// })) -// -// .apply(new ComputeSessions()) -// -// .apply(ParDo.named("SessionsToStrings").of(new SessionsToStringsDoFn())) -// .apply(new TopPerMonth()) -// .apply(ParDo.named("FormatOutput").of(new FormatOutputDoFn())); -// } -// } -// -// /** -// * Options supported by this class. -// * -// *

    Inherits standard Dataflow configuration options. -// */ -// private static interface Options extends PipelineOptions { -// @Description( -// "Input specified as a GCS path containing a BigQuery table exported as json") -// @Default.String(EXPORTED_WIKI_TABLE) -// String getInput(); -// void setInput(String value); -// -// @Description("File to output results to") -// @Validation.Required -// String getOutput(); -// void setOutput(String value); -// } -// -// public static void main(String[] args) { -// Options options = PipelineOptionsFactory.fromArgs(args) -// .withValidation() -// .as(Options.class); -// DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); -// -// Pipeline p = Pipeline.create(dataflowOptions); -// -// double samplingThreshold = 0.1; -// -// p.apply(TextIO.Read -// .from(options.getInput()) -// .withCoder(TableRowJsonCoder.of())) -// .apply(new ComputeTopSessions(samplingThreshold)) -// .apply(TextIO.Write.named("Write").withoutSharding().to(options.getOutput())); -// -// p.run(); -// } -//} diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java index 9c8147bd4a48..eb020c5798bf 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java @@ -1,71 +1,144 @@ -///* -// * Copyright 2015 Data Artisans GmbH -// * -// * Licensed 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 com.dataartisans.flink.dataflow; -// -//import com.google.api.services.bigquery.model.TableRow; -//import com.google.cloud.dataflow.sdk.Pipeline; -//import com.google.cloud.dataflow.sdk.io.TextIO; -//import com.google.cloud.dataflow.sdk.transforms.Create; -//import com.google.cloud.dataflow.sdk.values.PCollection; -//import com.google.common.base.Joiner; -//import org.apache.flink.test.util.JavaProgramTestBase; -// -//import java.util.Arrays; -// -//public class TopWikipediaSessionsITCase extends JavaProgramTestBase { -// protected String resultPath; -// -// public TopWikipediaSessionsITCase(){ -// } -// -// static final String[] EXPECTED_RESULT = new String[] { -// "user1 : [1970-01-01T00:00:00.000Z..1970-01-01T01:00:02.000Z)" -// + " : 3 : 1970-01-01T00:00:00.000Z", -// "user3 : [1970-02-05T00:00:00.000Z..1970-02-05T01:00:00.000Z)" -// + " : 1 : 1970-02-01T00:00:00.000Z" }; -// -// @Override -// protected void preSubmit() throws Exception { -// resultPath = getTempDirPath("result"); -// } -// -// @Override -// protected void postSubmit() throws Exception { -// compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); -// } -// -// @Override -// protected void testProgram() throws Exception { -// -// Pipeline p = FlinkTestPipeline.create(); -// -// PCollection output = -// p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", 0).set -// ("contributor_username", "user1"), new TableRow().set("timestamp", 1).set -// ("contributor_username", "user1"), new TableRow().set("timestamp", 2).set -// ("contributor_username", "user1"), new TableRow().set("timestamp", 0).set -// ("contributor_username", "user2"), new TableRow().set("timestamp", 1).set -// ("contributor_username", "user2"), new TableRow().set("timestamp", 3601).set -// ("contributor_username", "user2"), new TableRow().set("timestamp", 3602).set -// ("contributor_username", "user2"), new TableRow().set("timestamp", 35 * 24 * 3600) -// .set("contributor_username", "user3")))) -// .apply(new TopWikipediaSessions.ComputeTopSessions(1.0)); -// -// output.apply(TextIO.Write.to(resultPath)); -// -// p.run(); -// } -//} +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow; + +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.base.Joiner; +import org.apache.flink.streaming.util.StreamingProgramTestBase; +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.Serializable; +import java.util.Arrays; + + +/** + * Session window test + */ +public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable { + protected String resultPath; + + public TopWikipediaSessionsITCase(){ + } + + static final String[] EXPECTED_RESULT = new String[] { + "user: user1 value:3", + "user: user1 value:1", + "user: user2 value:4", + "user: user2 value:6", + "user: user3 value:7", + "user: user3 value:2" + }; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } + + @Override + protected void testProgram() throws Exception { + + Pipeline p = FlinkTestPipeline.createStreaming(); + + long now = System.currentTimeMillis() + 10000; + System.out.println((now + 5000) / 1000); + + PCollection> output = + p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set + ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now).set + ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set + ("contributor_username", "user1"), new TableRow().set("timestamp", now).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set + ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now) + .set("contributor_username", "user3")))) + + + + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + TableRow row = c.element(); + long timestamp = (Long) row.get("timestamp"); + String userName = (String) row.get("contributor_username"); + if (userName != null) { + // Sets the timestamp field to be used in windowing. + c.outputWithTimestamp(userName, new Instant(timestamp * 1000L)); + } + } + })) + + .apply(ParDo.named("SampleUsers").of( + new DoFn() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * 1.0) { + c.output(c.element()); + } + } + })) + + .apply(Window.into(Sessions.withGapDuration(Duration.standardMinutes(1)))) + .apply(Count.perElement()); + + PCollection format = output.apply(ParDo.of(new DoFn, String>() { + @Override + public void processElement(ProcessContext c) throws Exception { + KV el = c.element(); + String out = "user: " + el.getKey() + " value:" + el.getValue(); + System.out.println(out); + c.output(out); + } + })); + + format.apply(TextIO.Write.to(resultPath)); + + p.run(); + } +} From 46c5158642e8ff7ccbd5b11e6fce148eb21c28cf Mon Sep 17 00:00:00 2001 From: kl0u Date: Mon, 29 Feb 2016 12:38:56 +0100 Subject: [PATCH 133/149] Rearranging the code and renaming certain classes. --- ...=> FlinkPipelineExecutionEnvironment.java} | 104 ++++++++++-------- .../flink/dataflow/FlinkPipelineOptions.java | 23 +++- .../flink/dataflow/FlinkPipelineRunner.java | 6 +- .../examples/streaming/AutoComplete.java | 7 +- .../examples/streaming/JoinExamples.java | 5 +- .../KafkaWindowedWordCountExample.java | 3 + .../examples/streaming/WindowedWordCount.java | 3 + .../FlinkStreamingTransformTranslators.java | 2 - .../FlinkStreamingTranslationContext.java | 9 +- .../streaming/FlinkAbstractParDoWrapper.java | 10 +- .../FlinkGroupAlsoByWindowWrapper.java | 2 - .../io/FlinkStreamingCreateFunction.java | 7 +- .../streaming/io/UnboundedSourceWrapper.java | 11 +- .../flink/dataflow/AvroITCase.java | 4 +- .../flink/dataflow/FlattenizeITCase.java | 2 +- .../flink/dataflow/FlinkTestPipeline.java | 14 +-- .../flink/dataflow/JoinExamplesITCase.java | 2 +- .../flink/dataflow/MaybeEmptyTestITCase.java | 2 +- .../dataflow/ParDoMultiOutputITCase.java | 2 +- .../flink/dataflow/ReadSourceITCase.java | 2 +- .../dataflow/RemoveDuplicatesEmptyITCase.java | 2 +- .../dataflow/RemoveDuplicatesITCase.java | 2 +- .../flink/dataflow/SideInputITCase.java | 2 +- .../flink/dataflow/TfIdfITCase.java | 2 +- .../flink/dataflow/WordCountITCase.java | 2 +- .../flink/dataflow/WordCountJoin2ITCase.java | 2 +- .../flink/dataflow/WordCountJoin3ITCase.java | 2 +- .../flink/dataflow/WriteSinkITCase.java | 2 +- .../streaming/GroupAlsoByWindowTest.java | 8 +- .../TopWikipediaSessionsITCase.java | 5 +- 30 files changed, 145 insertions(+), 104 deletions(-) rename runners/flink/src/main/java/com/dataartisans/flink/dataflow/{FlinkJobExecutionEnvironment.java => FlinkPipelineExecutionEnvironment.java} (69%) rename runners/flink/src/test/java/com/dataartisans/flink/dataflow/{ => streaming}/TopWikipediaSessionsITCase.java (97%) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java similarity index 69% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java rename to runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java index 91b2f64d4bcb..a1372bd0cf25 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java @@ -19,6 +19,7 @@ import com.dataartisans.flink.dataflow.translation.FlinkBatchPipelineTranslator; import com.dataartisans.flink.dataflow.translation.FlinkStreamingPipelineTranslator; import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.common.base.Preconditions; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.CollectionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment; @@ -29,9 +30,9 @@ import java.util.List; -public class FlinkJobExecutionEnvironment { +public class FlinkPipelineExecutionEnvironment { - private static final Logger LOG = LoggerFactory.getLogger(FlinkJobExecutionEnvironment.class); + private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class); private final FlinkPipelineOptions options; @@ -49,50 +50,51 @@ public class FlinkJobExecutionEnvironment { * The Flink Streaming execution environment. This is instantiated to either a * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending - * on the configuration options, and more specifically, the url of the master url. + * on the configuration options, and more specifically, the url of the master. */ private StreamExecutionEnvironment flinkStreamEnv; /** - * Translator for this FlinkPipelineRunner. Its role is to translate the Dataflow operators to - * their Flink based counterparts. Based on the options provided by the user, if we have a streaming job, - * this is instantiated to a FlinkStreamingPipelineTranslator. In other case, i.e. a batch job, - * a FlinkBatchPipelineTranslator is created. + * Translator for this FlinkPipelineRunner. Its role is to translate the Beam operators to + * their Flink counterparts. Based on the options provided by the user, if we have a streaming job, + * this is instantiated as a {@link FlinkStreamingPipelineTranslator}. In other case, i.e. a batch job, + * a {@link FlinkBatchPipelineTranslator} is created. */ private FlinkPipelineTranslator flinkPipelineTranslator; - public FlinkJobExecutionEnvironment(FlinkPipelineOptions options) { - if (options == null) { - throw new IllegalArgumentException("Options in the FlinkJobExecutionEnvironment cannot be NULL."); - } - this.options = options; - this.createJobEnvironment(); - this.createJobGraphTranslator(); + /** + * Creates a {@link FlinkPipelineExecutionEnvironment} with the user-specified parameters in the + * provided {@link FlinkPipelineOptions}. + * + * @param options the user-defined pipeline options. + * */ + public FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) { + this.options = Preconditions.checkNotNull(options); + this.createPipelineExecutionEnvironment(); + this.createPipelineTranslator(); } /** * Depending on the type of job (Streaming or Batch) and the user-specified options, * this method creates the adequate ExecutionEnvironment. */ - private void createJobEnvironment() { + private void createPipelineExecutionEnvironment() { if (options.isStreaming()) { - LOG.info("Creating the required STREAMING Environment."); createStreamExecutionEnvironment(); } else { - LOG.info("Creating the required BATCH Environment."); createBatchExecutionEnvironment(); } } /** * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph - * translator. In the case of batch, it will work with DataSets, while for streaming, it will work - * with DataStreams. + * translator. In the case of batch, it will work with {@link org.apache.flink.api.java.DataSet}, + * while for streaming, it will work with {@link org.apache.flink.streaming.api.datastream.DataStream}. */ - private void createJobGraphTranslator() { + private void createPipelineTranslator() { checkInitializationState(); if (this.flinkPipelineTranslator != null) { - throw new IllegalStateException("JobGraphTranslator already initialized."); + throw new IllegalStateException("FlinkPipelineTranslator already initialized."); } this.flinkPipelineTranslator = options.isStreaming() ? @@ -100,35 +102,42 @@ private void createJobGraphTranslator() { new FlinkBatchPipelineTranslator(flinkBatchEnv, options); } + /** + * Depending on if the job is a Streaming or a Batch one, this method creates + * the necessary execution environment and pipeline translator, and translates + * the {@link com.google.cloud.dataflow.sdk.values.PCollection} program into + * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream} + * one. + * */ public void translate(Pipeline pipeline) { checkInitializationState(); if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) { - createJobEnvironment(); + createPipelineExecutionEnvironment(); } if (this.flinkPipelineTranslator == null) { - createJobGraphTranslator(); + createPipelineTranslator(); } this.flinkPipelineTranslator.translate(pipeline); } - public JobExecutionResult executeJob() throws Exception { + /** + * Launches the program execution. + * */ + public JobExecutionResult executePipeline() throws Exception { if (options.isStreaming()) { - - System.out.println("Plan: " + this.flinkStreamEnv.getExecutionPlan()); - if (this.flinkStreamEnv == null) { - throw new RuntimeException("JobExecutionEnvironment not initialized."); + throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized."); } if (this.flinkPipelineTranslator == null) { - throw new RuntimeException("JobGraphTranslator not initialized."); + throw new RuntimeException("FlinkPipelineTranslator not initialized."); } return this.flinkStreamEnv.execute(); } else { if (this.flinkBatchEnv == null) { - throw new RuntimeException("JobExecutionEnvironment not initialized."); + throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized."); } if (this.flinkPipelineTranslator == null) { - throw new RuntimeException("JobGraphTranslator not initialized."); + throw new RuntimeException("FlinkPipelineTranslator not initialized."); } return this.flinkBatchEnv.execute(); } @@ -141,9 +150,11 @@ public JobExecutionResult executeJob() throws Exception { */ private void createBatchExecutionEnvironment() { if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) { - throw new RuntimeException("JobExecutionEnvironment already initialized."); + throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized."); } + LOG.info("Creating the required Batch Execution Environment."); + String masterUrl = options.getFlinkMaster(); this.flinkStreamEnv = null; @@ -181,9 +192,11 @@ private void createBatchExecutionEnvironment() { */ private void createStreamExecutionEnvironment() { if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) { - throw new RuntimeException("JobExecutionEnvironment already initialized."); + throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized."); } + LOG.info("Creating the required Streaming Environment."); + String masterUrl = options.getFlinkMaster(); this.flinkBatchEnv = null; @@ -213,21 +226,26 @@ private void createStreamExecutionEnvironment() { // although we do not use the generated timestamps, // enabling timestamps is needed for the watermarks. this.flinkStreamEnv.getConfig().enableTimestamps(); - this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); - this.flinkStreamEnv.enableCheckpointing(1000); - this.flinkStreamEnv.setNumberOfExecutionRetries(5); - - LOG.info("Setting execution retry delay to 3 sec"); - this.flinkStreamEnv.getConfig().setExecutionRetryDelay(3000); + // for the following 2 parameters, a value of -1 means that Flink will use + // the default values as specified in the configuration. + this.flinkStreamEnv.setNumberOfExecutionRetries(options.getNumberOfExecutionRetries()); + this.flinkStreamEnv.getConfig().setExecutionRetryDelay(options.getExecutionRetryDelay()); + + // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink). + // If the value is not -1, then the validity checks are applied. + // By default, checkpointing is disabled. + long checkpointInterval = options.getCheckpointingInterval(); + if(checkpointInterval != -1) { + if (checkpointInterval < 1) { + throw new IllegalArgumentException("The checkpoint interval must be positive"); + } + this.flinkStreamEnv.enableCheckpointing(checkpointInterval); + } } private void checkInitializationState() { - if (this.options == null) { - throw new IllegalStateException("FlinkJobExecutionEnvironment is not initialized yet."); - } - if (options.isStreaming() && this.flinkBatchEnv != null) { throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment."); } else if (!options.isStreaming() && this.flinkStreamEnv != null) { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java index e746f414495b..2429caca8153 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java @@ -66,11 +66,26 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp String getFlinkMaster(); void setFlinkMaster(String value); - /** - * The degree of parallelism to be used when parallelizing operations onto workers. - */ - @Description("The degree of parallelism to be used when parallelizing operations onto workers.") + @Description("The degree of parallelism to be used when distributing operations onto workers.") @Default.Integer(-1) Integer getParallelism(); void setParallelism(Integer value); + + @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " + + "fault tolerance).") + @Default.Long(-1L) + Long getCheckpointingInterval(); + void setCheckpointingInterval(Long interval); + + @Description("Sets the number of times that failed tasks are re-executed. " + + "A value of zero effectively disables fault tolerance. A value of -1 indicates " + + "that the system default value (as defined in the configuration) should be used.") + @Default.Integer(-1) + Integer getNumberOfExecutionRetries(); + void setNumberOfExecutionRetries(Integer retries); + + @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.") + @Default.Long(-1L) + Long getExecutionRetryDelay(); + void setExecutionRetryDelay(Long delay); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java index ebd2691a7529..7ea837016aca 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java @@ -54,7 +54,7 @@ public class FlinkPipelineRunner extends PipelineRunner { */ private final FlinkPipelineOptions options; - private final FlinkJobExecutionEnvironment flinkJobEnv; + private final FlinkPipelineExecutionEnvironment flinkJobEnv; /** * Construct a runner from the provided options. @@ -103,7 +103,7 @@ public static FlinkPipelineRunner fromOptions(PipelineOptions options) { private FlinkPipelineRunner(FlinkPipelineOptions options) { this.options = options; - this.flinkJobEnv = new FlinkJobExecutionEnvironment(options); + this.flinkJobEnv = new FlinkPipelineExecutionEnvironment(options); } @Override @@ -118,7 +118,7 @@ public FlinkRunnerResult run(Pipeline pipeline) { JobExecutionResult result; try { - result = this.flinkJobEnv.executeJob(); + result = this.flinkJobEnv.executePipeline(); } catch (Exception e) { LOG.error("Pipeline execution failed", e); throw new RuntimeException("Pipeline execution failed", e); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java index 711d9fbc1612..493fb25c3b80 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java @@ -325,7 +325,9 @@ public void processElement(ProcessContext c) { * Takes as input a the top candidates per prefix, and emits an entity * suitable for writing to Datastore. */ - static class FormatForPerTaskLocalFile extends DoFn>, String> { + static class FormatForPerTaskLocalFile extends DoFn>, String> + implements DoFn.RequiresWindowAccess{ + private static final long serialVersionUID = 0; @Override @@ -357,6 +359,9 @@ private interface Options extends WindowedWordCount.StreamingWordCountOptions { public static void main(String[] args) throws IOException { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); options.setStreaming(true); + options.setCheckpointingInterval(1000L); + options.setNumberOfExecutionRetries(5); + options.setExecutionRetryDelay(3000L); options.setRunner(FlinkPipelineRunner.class); PTransform> readSource = diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java index 9a5db6463ae9..60f6788561cd 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java @@ -124,9 +124,10 @@ private interface Options extends WindowedWordCount.StreamingWordCountOptions { public static void main(String[] args) throws Exception { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - - // make it a streaming example. options.setStreaming(true); + options.setCheckpointingInterval(1000L); + options.setNumberOfExecutionRetries(5); + options.setExecutionRetryDelay(3000L); options.setRunner(FlinkPipelineRunner.class); PTransform> readSourceA = diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java index 42d3d883b2c0..dba2721c5afb 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java @@ -104,6 +104,9 @@ public static void main(String[] args) { KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class); options.setJobName("KafkaExample"); options.setStreaming(true); + options.setCheckpointingInterval(1000L); + options.setNumberOfExecutionRetries(5); + options.setExecutionRetryDelay(3000L); options.setRunner(FlinkPipelineRunner.class); System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() ); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java index b539245cf3f3..37dc39a6cac5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java @@ -99,6 +99,9 @@ public static void main(String[] args) throws IOException { options.setStreaming(true); options.setWindowSize(10L); options.setSlide(5L); + options.setCheckpointingInterval(1000L); + options.setNumberOfExecutionRetries(5); + options.setExecutionRetryDelay(3000L); options.setRunner(FlinkPipelineRunner.class); LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() + diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java index 46d3e36551df..27cc923b3c74 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java @@ -69,7 +69,6 @@ public class FlinkStreamingTransformTranslators { // here you can find all the available translators. static { - TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator()); TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator()); TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator()); @@ -79,7 +78,6 @@ public class FlinkStreamingTransformTranslators { TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator()); TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator()); - } public static FlinkStreamingPipelineTranslator.StreamTransformTranslator getTranslator(PTransform transform) { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java index df68e5072afd..7c4ab93dbbe5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java @@ -18,7 +18,10 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.values.*; +import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.common.base.Preconditions; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -40,8 +43,8 @@ public class FlinkStreamingTranslationContext { private AppliedPTransform currentTransform; public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) { - this.env = env; - this.options = options; + this.env = Preconditions.checkNotNull(env); + this.options = Preconditions.checkNotNull(options); this.dataStreams = new HashMap<>(); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index 71f9c7f089a2..dfb2b7de2a49 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -116,10 +116,10 @@ public Instant timestamp() { @Override public BoundedWindow window() { -// if (!(fn instanceof DoFn.RequiresWindowAccess)) { -// throw new UnsupportedOperationException( -// "window() is only available in the context of a DoFn marked as RequiresWindow."); -// } + if (!(fn instanceof DoFn.RequiresWindowAccess)) { + throw new UnsupportedOperationException( + "window() is only available in the context of a DoFn marked as RequiresWindow."); + } Collection windows = this.element.getWindows(); if (windows.size() != 1) { @@ -211,7 +211,7 @@ protected WindowedValue makeWindowedValue( @Override public Object element() { throw new UnsupportedOperationException( - "WindowFn attempted to access input element when none was available"); // TODO: 12/16/15 aljoscha's comment in slack + "WindowFn attempted to access input element when none was available"); } @Override diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java index 0f0a9d0daace..b78db65a9b58 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -238,9 +238,7 @@ private StreamingGroupAlsoByWindowsDoFn createGroupAlsoByWindowOperator() { this.operator = StreamingGroupAlsoByWindowsDoFn.createForIterable( this.windowingStrategy, inputValueCoder); } else { - Coder inputKeyCoder = inputKvCoder.getKeyCoder(); - //CoderRegistry dataflowRegistry = input.getPipeline().getCoderRegistry(); AppliedCombineFn appliedCombineFn = AppliedCombineFn .withInputCoder(combineFn, coderRegistry, inputKvCoder); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java index b8824f5763f7..c952d6f0bdf9 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.util.WindowedValue; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; +import org.joda.time.Instant; import java.io.ByteArrayInputStream; import java.util.List; @@ -44,17 +45,15 @@ public FlinkStreamingCreateFunction(List elements, Coder coder) { public void flatMap(IN value, Collector> out) throws Exception { @SuppressWarnings("unchecked") - // TODO Flink doesn't allow null values in records OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE; - for (byte[] element : elements) { ByteArrayInputStream bai = new ByteArrayInputStream(element); OUT outValue = coder.decode(bai, Coder.Context.OUTER); if (outValue == null) { - out.collect(WindowedValue.of(voidValue, GlobalWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } else { - out.collect(WindowedValue.of(outValue, GlobalWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index 3e248a662597..cdc2e954cee4 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -38,7 +38,7 @@ public class UnboundedSourceWrapper extends RichSourceFunction reader; private StreamingRuntimeContext runtime = null; - private StreamSource.ManualWatermarkContext context = null; + private StreamSource.ManualWatermarkContext> context = null; private volatile boolean isRunning = false; @@ -51,8 +51,7 @@ public String getName() { return this.name; } - WindowedValue makeWindowedValue( - T output, Instant timestamp, Collection windows, PaneInfo pane) { + WindowedValue makeWindowedValue(T output, Instant timestamp) { if (timestamp == null) { timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; } @@ -66,7 +65,7 @@ public void run(SourceContext> ctx) throws Exception { "Apparently " + this.name + " is not. Probably you should consider writing your own Wrapper for this source."); } - context = (StreamSource.ManualWatermarkContext) ctx; + context = (StreamSource.ManualWatermarkContext>) ctx; runtime = (StreamingRuntimeContext) getRuntimeContext(); this.isRunning = reader.start(); @@ -78,11 +77,9 @@ public void run(SourceContext> ctx) throws Exception { T item = reader.getCurrent(); Instant timestamp = reader.getCurrentTimestamp(); - long milliseconds = timestamp.getMillis(); - // write it to the output collector synchronized (ctx.getCheckpointLock()) { - ctx.collectWithTimestamp(makeWindowedValue(item, timestamp, null, PaneInfo.NO_FIRING), milliseconds); + context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis()); } // try to go to the next record diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java index c6e3e99a6e93..2b1f091fe5ca 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java @@ -56,14 +56,14 @@ protected void testProgram() throws Exception { } private static void runProgram(String tmpPath, String resultPath) { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); p.apply(Create.of(new User("Joe", 3, "red"), new User("Mary", 4, "blue")).withCoder(AvroCoder.of(User.class))) .apply(AvroIO.Write.to(tmpPath).withSchema(User.class)); p.run(); - p = FlinkTestPipeline.create(); + p = FlinkTestPipeline.createForBatch(); p.apply(AvroIO.Read.from(tmpPath).withSchema(User.class)) .apply(ParDo.of(new DoFn() { diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java index bc2451498a42..928388cea011 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java @@ -51,7 +51,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); PCollection p1 = p.apply(Create.of(words)); PCollection p2 = p.apply(Create.of(words2)); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java index 109b1ff8e75a..56af3f1eb20f 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java @@ -32,7 +32,7 @@ public class FlinkTestPipeline extends Pipeline { *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call * {@link Pipeline#run} to execute the pipeline and check the tests. */ - public static FlinkTestPipeline create() { + public static FlinkTestPipeline createForBatch() { return create(false); } @@ -44,7 +44,7 @@ public static FlinkTestPipeline create() { * * @return The Test Pipeline */ - public static FlinkTestPipeline createStreaming() { + public static FlinkTestPipeline createForStreaming() { return create(true); } @@ -54,18 +54,18 @@ public static FlinkTestPipeline createStreaming() { *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call * {@link Pipeline#run} to execute the pipeline and check the tests. * - * @param streaming True for streaming mode, False for batch - * @return The Test Pipeline + * @param streaming True for streaming mode, False for batch. + * @return The Test Pipeline. */ - public static FlinkTestPipeline create(boolean streaming) { + private static FlinkTestPipeline create(boolean streaming) { FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming); FlinkPipelineOptions pipelineOptions = flinkRunner.getPipelineOptions(); pipelineOptions.setStreaming(streaming); return new FlinkTestPipeline(flinkRunner, pipelineOptions); } - private FlinkTestPipeline(PipelineRunner runner, PipelineOptions - options) { + private FlinkTestPipeline(PipelineRunner runner, + PipelineOptions options) { super(runner, options); } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java index ed2ecf546406..af0f21779157 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java @@ -84,7 +84,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); PCollection input1 = p.apply(Create.of(EVENT_ARRAY)); PCollection input2 = p.apply(Create.of(CC_ARRAY)); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java index 29c34d4b3365..35f2eaff463d 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java @@ -47,7 +47,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); p.apply(Create.of((Void) null)).setCoder(VoidCoder.of()) .apply(ParDo.of( diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java index dbe88d2cfbcf..ccdbbf90b134 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java @@ -47,7 +47,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); PCollection words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO")); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java index ba675b1bb92f..39f54e44f851 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java @@ -61,7 +61,7 @@ protected void testProgram() throws Exception { private static void runProgram(String resultPath) { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); PCollection result = p .apply(Read.from(new ReadSource(1, 10))) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java index ff59db7e31bc..db794f73da75 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java @@ -52,7 +52,7 @@ protected void testProgram() throws Exception { List strings = Collections.emptyList(); - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); PCollection input = p.apply(Create.of(strings)) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java index a8200aa4c12d..04e06b88118d 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java @@ -53,7 +53,7 @@ protected void testProgram() throws Exception { List strings = Arrays.asList("k1", "k5", "k5", "k2", "k1", "k2", "k3"); - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); PCollection input = p.apply(Create.of(strings)) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java index d932c80ffb71..ee8843c5e8da 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java @@ -36,7 +36,7 @@ public class SideInputITCase extends JavaProgramTestBase implements Serializable protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); final PCollectionView sidesInput = p diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java index e801ac434540..1b4afb3f9d19 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java @@ -53,7 +53,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - Pipeline pipeline = FlinkTestPipeline.create(); + Pipeline pipeline = FlinkTestPipeline.createForBatch(); pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java index 9427ab681196..5ddd379fd095 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java @@ -58,7 +58,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); PCollection input = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java index c3eed61d774b..ccc52c43b3ea 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java @@ -70,7 +70,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); /* Create two PCollections and join them */ PCollection> occurences1 = p.apply(Create.of(WORDS_1)) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java index 33e67cc234e8..e6eddc0c6a55 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java @@ -80,7 +80,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); /* Create two PCollections and join them */ PCollection> occurences1 = p.apply(Create.of(WORDS_1)) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java index 205fe9bddc2d..865fc5f93894 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java @@ -63,7 +63,7 @@ protected void testProgram() throws Exception { } private static void runProgram(String resultPath) { - Pipeline p = FlinkTestPipeline.create(); + Pipeline p = FlinkTestPipeline.createForBatch(); p.apply(Create.of(EXPECTED_RESULT)).setCoder(StringUtf8Coder.of()) .apply("CustomSink", Write.to(new MyCustomSink(resultPath))); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java index b6671876c547..1f36ee72b9bd 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java @@ -86,7 +86,7 @@ public void testWithLateness() throws Exception { .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES) .withAllowedLateness(Duration.millis(1000)); long initialTime = 0L; - Pipeline pipeline = FlinkTestPipeline.create(); + Pipeline pipeline = FlinkTestPipeline.createForStreaming(); KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); @@ -145,7 +145,7 @@ public void testSessionWindows() throws Exception { WindowingStrategy strategy = sessionWindowingStrategy; long initialTime = 0L; - Pipeline pipeline = FlinkTestPipeline.create(); + Pipeline pipeline = FlinkTestPipeline.createForStreaming(); KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); @@ -382,7 +382,7 @@ public void testCompoundAccumulatingPanesProgram() throws Exception { } private OneInputStreamOperatorTestHarness createTestingOperatorAndState(WindowingStrategy strategy, long initialTime) throws Exception { - Pipeline pipeline = FlinkTestPipeline.create(); + Pipeline pipeline = FlinkTestPipeline.createForStreaming(); KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); @@ -478,7 +478,7 @@ private WindowedValue makeWindowedValue(WindowingStrategy strategy, @Override public Object element() { throw new UnsupportedOperationException( - "WindowFn attempted to access input element when none was available"); // TODO: 12/16/15 aljoscha's comment in slack + "WindowFn attempted to access input element when none was available"); } @Override diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java similarity index 97% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java rename to runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java index eb020c5798bf..1c800fa1ac0c 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java @@ -13,8 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package com.dataartisans.flink.dataflow.streaming; +import com.dataartisans.flink.dataflow.FlinkTestPipeline; import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; @@ -66,7 +67,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.createStreaming(); + Pipeline p = FlinkTestPipeline.createForStreaming(); long now = System.currentTimeMillis() + 10000; System.out.println((now + 5000) / 1000); From 0cb6deb105a56b85c6de5d9ed4379e655455c78b Mon Sep 17 00:00:00 2001 From: kl0u Date: Mon, 29 Feb 2016 12:59:48 +0100 Subject: [PATCH 134/149] Adds javadocs. --- .../dataflow/FlinkPipelineExecutionEnvironment.java | 8 ++++++++ .../translation/FlinkPipelineTranslator.java | 9 +++++++++ .../FlinkStreamingPipelineTranslator.java | 13 +++++++++++-- .../FlinkStreamingTransformTranslators.java | 9 ++++----- .../streaming/FlinkAbstractParDoWrapper.java | 5 +++++ .../streaming/FlinkGroupAlsoByWindowWrapper.java | 6 +++--- .../wrappers/streaming/FlinkGroupByKeyWrapper.java | 4 ++++ .../streaming/FlinkParDoBoundMultiWrapper.java | 3 +++ .../wrappers/streaming/FlinkParDoBoundWrapper.java | 3 +++ .../wrappers/streaming/io/UnboundedFlinkSource.java | 4 ++++ .../streaming/io/UnboundedSocketSource.java | 3 +++ .../streaming/io/UnboundedSourceWrapper.java | 9 +++++++-- .../state/AbstractFlinkTimerInternals.java | 4 ++++ .../streaming/state/FlinkStateInternals.java | 4 ++++ .../wrappers/streaming/state/StateType.java | 4 ++++ 15 files changed, 76 insertions(+), 12 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java index a1372bd0cf25..09ca18447451 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java @@ -30,6 +30,14 @@ import java.util.List; +/** + * The class that instantiates and manages the execution of a given job. + * Depending on if the job is a Streaming or Batch processing one, it creates + * the adequate execution environment ({@link ExecutionEnvironment} or {@link StreamExecutionEnvironment}), + * the necessary {@link FlinkPipelineTranslator} ({@link FlinkBatchPipelineTranslator} or + * {@link FlinkStreamingPipelineTranslator})to transform the Beam job into a Flink one, and + * executes the (translated) job. + */ public class FlinkPipelineExecutionEnvironment { private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java index e5c85451b2b6..b56fe0707326 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java @@ -17,6 +17,15 @@ import com.google.cloud.dataflow.sdk.Pipeline; +/** + * The role of this class is to translate the Beam operators to + * their Flink counterparts. If we have a streaming job, this is instantiated as a + * {@link FlinkStreamingPipelineTranslator}. In other case, i.e. for a batch job, + * a {@link FlinkBatchPipelineTranslator} is created. Correspondingly, the + * {@link com.google.cloud.dataflow.sdk.values.PCollection}-based user-provided job is translated into + * a {@link org.apache.flink.streaming.api.datastream.DataStream} (for streaming) or a + * {@link org.apache.flink.api.java.DataSet} (for batch) one. + */ public abstract class FlinkPipelineTranslator implements Pipeline.PipelineVisitor { public void translate(Pipeline pipeline) { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java index a8f42263e6aa..ea9ed140cff5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java @@ -22,6 +22,13 @@ import com.google.cloud.dataflow.sdk.values.PValue; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +/** + * This is a {@link FlinkPipelineTranslator} for streaming jobs. Its role is to translate the user-provided + * {@link com.google.cloud.dataflow.sdk.values.PCollection}-based job into a + * {@link org.apache.flink.streaming.api.datastream.DataStream} one. + * + * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator} + * */ public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator { /** The necessary context in the case of a straming job. */ @@ -107,14 +114,16 @@ public void visitValue(PValue value, TransformTreeNode producer) { @SuppressWarnings("unchecked") StreamTransformTranslator typedTranslator = (StreamTransformTranslator) translator; - // create the applied PTransform on the batchContext + // create the applied PTransform on the streamingContext streamingContext.setCurrentTransform(AppliedPTransform.of( node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); typedTranslator.translateNode(typedTransform, streamingContext); } /** - * A translator of a {@link PTransform}. + * The interface that every Flink translator of a Beam operator should implement. + * This interface is for streaming jobs. For examples of such translators see + * {@link FlinkStreamingTransformTranslators}. */ public interface StreamTransformTranslator { void translateNode(Type transform, FlinkStreamingTranslationContext context); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java index 27cc923b3c74..1be51aee4787 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java @@ -52,11 +52,10 @@ import java.util.*; /** - *

    - * Coder entryCoder = pCollection.getCoder(); - * if (!(entryCoder instanceof KvCoder)) { - * throw new IllegalArgumentException("PCollection does not use a KvCoder"); - * } + * This class contains all the mappings between Beam and Flink + * streaming transformations. The {@link FlinkStreamingPipelineTranslator} + * traverses the Beam job and comes here to translate the encountered Beam transformations + * into Flink one, based on the mapping available in this class. */ public class FlinkStreamingTransformTranslators { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index dfb2b7de2a49..3605d3ff0117 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -37,6 +37,11 @@ import java.util.Collection; +/** + * An abstract class that encapsulates the common code of the the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.Bound} + * and {@link com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti} wrappers. See the {@link FlinkParDoBoundWrapper} and + * {@link FlinkParDoBoundMultiWrapper} for the actual wrappers of the aforementioned transformations. + * */ public abstract class FlinkAbstractParDoWrapper extends RichFlatMapFunction, WindowedValue> { private final DoFn doFn; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java index b78db65a9b58..75694cc2ce92 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -46,9 +46,9 @@ import java.util.*; /** - * This class is the key class implementing all the windowing/triggering logic of Google Dataflow. - * To provide full compatibility and support all the windowing/triggering combinations offered by - * Dataflow, we opted for a strategy that uses the SDK's code for doing these operations + * This class is the key class implementing all the windowing/triggering logic of Apache Beam. + * To provide full compatibility and support for all the windowing/triggering combinations offered by + * Beam, we opted for a strategy that uses the SDK's code for doing these operations. See the code in * ({@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn}. *

    * In a nutshell, when the execution arrives to this operator, we expect to have a stream already diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java index 0a0e3012b401..b0d9e48dd608 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java @@ -26,6 +26,10 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.KeyedStream; +/** + * This class groups the elements by key. It assumes that already the incoming stream + * is composed of [Key,Value] pairs. + * */ public class FlinkGroupByKeyWrapper { /** diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java index 200c397dff15..52ab19efeb94 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java @@ -28,6 +28,9 @@ import java.util.Map; +/** + * A wrapper for the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti} Beam transformation. + * */ public class FlinkParDoBoundMultiWrapper extends FlinkAbstractParDoWrapper { private final TupleTag mainTag; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java index 18d42494fc04..4a5c8548832c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java @@ -29,6 +29,9 @@ import java.io.IOException; import java.util.*; +/** + * A wrapper for the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.Bound} Beam transformation. + * */ public class FlinkParDoBoundWrapper extends FlinkAbstractParDoWrapper { public FlinkParDoBoundWrapper(PipelineOptions options, WindowingStrategy windowingStrategy, DoFn doFn) { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java index 17e0746fdbae..7c8cd0b58640 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java @@ -25,6 +25,10 @@ import javax.annotation.Nullable; import java.util.List; +/** + * A wrapper translating Flink Sources implementing the {@link RichParallelSourceFunction} interface, into + * unbounded Beam sources (see {@link UnboundedSource}). + * */ public class UnboundedFlinkSource extends UnboundedSource { private final PipelineOptions options; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java index 2b0d6dc4a866..dd14f68867fe 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java @@ -36,6 +36,9 @@ import static com.google.common.base.Preconditions.checkArgument; +/** + * An example unbounded Beam source that reads input from a socket. This is used mainly for testing and debugging. + * */ public class UnboundedSocketSource extends UnboundedSource { private static final Coder DEFAULT_SOCKET_CODER = StringUtf8Coder.of(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index cdc2e954cee4..c534079dccd8 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -30,8 +30,13 @@ import org.apache.flink.streaming.runtime.operators.Triggerable; import org.joda.time.Instant; -import java.util.Collection; - +/** + * A wrapper for Beam's unbounded sources. This class wraps around a source implementing the {@link com.google.cloud.dataflow.sdk.io.Read.Unbounded} + * interface. + * + *

    + * For now we support non-parallel, not checkpointed sources. + * */ public class UnboundedSourceWrapper extends RichSourceFunction> implements EventTimeSourceFunction>, Triggerable { private final String name; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java index 4401eb372f5c..7b68e9fb67dc 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java @@ -29,6 +29,10 @@ import java.io.IOException; import java.io.Serializable; +/** + * An implementation of Beam's {@link TimerInternals}, that also provides serialization functionality. + * The latter is used when snapshots of the current state are taken, for fault-tolerance. + * */ public abstract class AbstractFlinkTimerInternals implements TimerInternals, Serializable { private TimerOrElement>> element; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java index 03b8bb5be4c0..f4ec6d592697 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java @@ -28,6 +28,10 @@ import java.io.IOException; import java.util.*; +/** + * An implementation of the Beam {@link MergingStateInternals}. This implementation simply keeps elements in memory. + * This state is periodically checkpointed by Flink, for fault-tolerance. + * */ public class FlinkStateInternals extends MergingStateInternals { private final K key; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java index 11446ea48c3c..aa049efbacc3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java @@ -17,6 +17,10 @@ import java.io.IOException; +/** + * The available types of state, as provided by the Beam SDK. This class is used for serialization/deserialization + * purposes. + * */ public enum StateType { VALUE(0), From 13e027626e9c5faf82bfba292d7a82b9ca763a0b Mon Sep 17 00:00:00 2001 From: kl0u Date: Mon, 29 Feb 2016 16:26:12 +0100 Subject: [PATCH 135/149] Fixes Void handling --- .../streaming/FlinkGroupByKeyWrapper.java | 8 +- .../flink/dataflow/FlinkTestPipeline.java | 4 +- .../streaming/GroupByNullKeyTest.java | 121 ++++++++++++++++++ 3 files changed, 128 insertions(+), 5 deletions(-) create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java index b0d9e48dd608..24f6d40999c1 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java @@ -16,9 +16,11 @@ package com.dataartisans.flink.dataflow.translation.wrappers.streaming; import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; +import com.dataartisans.flink.dataflow.translation.types.VoidCoderTypeSerializer; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; -import com.google.cloud.dataflow.sdk.util.*; +import com.google.cloud.dataflow.sdk.coders.VoidCoder; +import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.KV; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; @@ -42,13 +44,15 @@ private interface KeySelectorWithQueryableResultType extends KeySelector KeyedStream>, K> groupStreamByKey(DataStream>> inputDataStream, KvCoder inputKvCoder) { final Coder keyCoder = inputKvCoder.getKeyCoder(); final TypeInformation keyTypeInfo = new CoderTypeInformation<>(keyCoder); + final boolean isKeyVoid = keyCoder instanceof VoidCoder; return inputDataStream.keyBy( new KeySelectorWithQueryableResultType() { @Override public K getKey(WindowedValue> value) throws Exception { - return value.getValue().getKey(); + return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE : + value.getValue().getKey(); } @Override diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java index 56af3f1eb20f..59c3b69998ce 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java @@ -59,9 +59,7 @@ public static FlinkTestPipeline createForStreaming() { */ private static FlinkTestPipeline create(boolean streaming) { FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming); - FlinkPipelineOptions pipelineOptions = flinkRunner.getPipelineOptions(); - pipelineOptions.setStreaming(streaming); - return new FlinkTestPipeline(flinkRunner, pipelineOptions); + return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions()); } private FlinkTestPipeline(PipelineRunner runner, diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java new file mode 100644 index 000000000000..5a412aa54098 --- /dev/null +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java @@ -0,0 +1,121 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow.streaming; + +import com.dataartisans.flink.dataflow.FlinkTestPipeline; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.GroupByKey; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark; +import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.base.Joiner; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.util.StreamingProgramTestBase; +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.Serializable; +import java.util.Arrays; + +public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable { + + + protected String resultPath; + + static final String[] EXPECTED_RESULT = new String[] { + "k: null v: user1 user1 user1 user2 user2 user2 user2 user3" + }; + + public GroupByNullKeyTest(){ + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } + + public static class ExtractUserAndTimestamp extends DoFn, String> { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + KV record = c.element(); + long now = System.currentTimeMillis(); + int timestamp = record.getKey(); + String userName = record.getValue(); + if (userName != null) { + // Sets the implicit timestamp field to be used in windowing. + c.outputWithTimestamp(userName, new Instant(timestamp + now)); + } + } + } + + @Override + protected void testProgram() throws Exception { + + Pipeline p = FlinkTestPipeline.createForStreaming(); + + PCollection output = + p.apply(Create.of(Arrays.asList( + KV.of(0, "user1"), + KV.of(1, "user1"), + KV.of(2, "user1"), + KV.of(10, "user2"), + KV.of(1, "user2"), + KV.of(15000, "user2"), + KV.of(12000, "user2"), + KV.of(25000, "user3")))) + .apply(ParDo.of(new ExtractUserAndTimestamp())) + .apply(Window.into(FixedWindows.of(Duration.standardHours(1))) + .triggering(AfterWatermark.pastEndOfWindow()) + .withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()) + + .apply(ParDo.of(new DoFn>() { + @Override + public void processElement(ProcessContext c) throws Exception { + String elem = c.element(); + c.output(KV.of((Void) null, elem)); + } + })) + .apply(GroupByKey.create()) + .apply(ParDo.of(new DoFn>, String>() { + @Override + public void processElement(ProcessContext c) throws Exception { + KV> elem = c.element(); + StringBuilder str = new StringBuilder(); + str.append("k: " + elem.getKey() + " v:"); + for (String v : elem.getValue()) { + str.append(" " + v); + } + c.output(str.toString()); + } + })); + output.apply(TextIO.Write.to(resultPath)); + p.run(); + } +} From 3c3a6616cef1e348199c0d884eda5c2b929e4b16 Mon Sep 17 00:00:00 2001 From: Mark Shields Date: Wed, 24 Feb 2016 06:54:51 -0800 Subject: [PATCH 136/149] Track dataflow 1.0 -> 1.5 changes --- runners/flink/.gitignore | 1 + runners/flink/pom.xml | 114 +++-- .../FlinkPipelineExecutionEnvironment.java | 2 +- .../flink/dataflow/FlinkPipelineOptions.java | 3 +- .../flink/dataflow/examples/WordCount.java | 54 ++- .../FlinkBatchTransformTranslators.java | 2 +- .../functions/FlinkCreateFunction.java | 2 + .../functions/FlinkDoFnFunction.java | 4 + .../types/CoderTypeInformation.java | 3 +- .../types/KvCoderTypeInformation.java | 2 +- .../wrappers/SinkOutputFormat.java | 2 +- .../streaming/FlinkAbstractParDoWrapper.java | 5 +- .../FlinkGroupAlsoByWindowWrapper.java | 257 ++++++----- .../FlinkParDoBoundMultiWrapper.java | 2 +- .../streaming/FlinkParDoBoundWrapper.java | 6 + .../io/FlinkStreamingCreateFunction.java | 3 + .../streaming/io/UnboundedFlinkSource.java | 2 +- .../streaming/io/UnboundedSourceWrapper.java | 4 +- .../state/AbstractFlinkTimerInternals.java | 125 +++--- .../streaming/state/FlinkStateInternals.java | 401 +++++++++++++----- .../streaming/state/StateCheckpointUtils.java | 5 +- .../state/StateCheckpointWriter.java | 8 +- .../flink/dataflow/AvroITCase.java | 83 ---- .../flink/dataflow/ReadSourceITCase.java | 2 +- .../flink/dataflow/TfIdfITCase.java | 4 +- .../flink/dataflow/WordCountITCase.java | 5 +- .../streaming/GroupAlsoByWindowTest.java | 23 +- .../streaming/StateSerializationTest.java | 172 +++++--- .../streaming/TopWikipediaSessionsITCase.java | 7 +- 29 files changed, 756 insertions(+), 547 deletions(-) delete mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java diff --git a/runners/flink/.gitignore b/runners/flink/.gitignore index 218a1185da17..e40b671f8af5 100644 --- a/runners/flink/.gitignore +++ b/runners/flink/.gitignore @@ -16,3 +16,4 @@ tmp *.log .DS_Store _site +\? \ No newline at end of file diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index cb784a080bc0..acf79d19ab78 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -3,29 +3,29 @@ Copyright 2015 Data Artisans GmbH - Licensed 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 + Licensed 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 + 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. + 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. --> + xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> 4.0.0 com.dataartisans flink-dataflow - 0.2 + 0.3-SNAPSHOT - Flink Dataflow Runner + Flink Beam Runner jar 2015 @@ -41,12 +41,13 @@ UTF-8 UTF-8 - 0.10.1 - - com.dataartisans.flink.dataflow.examples.WordCount - kinglear.txt - wordcounts.txt - 1 + 1.0-SNAPSHOT + 1.5.0-SNAPSHOT + + com.dataartisans.flink.dataflow.examples.WordCount + kinglear.txt + wordcounts.txt + 1 @@ -56,9 +57,11 @@ https://repository.apache.org/content/repositories/snapshots/ false + never - true + false + never @@ -71,7 +74,12 @@ org.apache.flink - flink-streaming-java + flink-streaming-java_2.10 + ${flink.version} + + + org.apache.flink + flink-streaming-java_2.10 ${flink.version} test test-jar @@ -83,37 +91,29 @@ org.apache.flink - flink-avro + flink-avro_2.10 ${flink.version} org.apache.flink - flink-clients + flink-clients_2.10 ${flink.version} org.apache.flink - flink-test-utils + flink-test-utils_2.10 ${flink.version} test - - com.google.cloud.dataflow - google-cloud-dataflow-java-sdk-all - 1.0.0 - - - org.slf4j - slf4j-jdk14 - - + org.apache.flink + flink-connector-kafka-0.8_2.10 + ${flink.version} - com.google.cloud.dataflow - google-cloud-dataflow-java-examples-all - 1.0.0 + google-cloud-dataflow-java-sdk-all + ${beam.version} org.slf4j @@ -121,11 +121,6 @@ - - org.apache.flink - flink-connector-kafka - ${flink.version} - org.mockito mockito-all @@ -133,19 +128,20 @@ test - org.apache.flink - flink-streaming-java - ${flink.version} + com.google.guava + guava + 19.0 + org.apache.maven.plugins maven-jar-plugin - 2.4 + 2.6 @@ -230,20 +226,20 @@ - org.codehaus.mojo - exec-maven-plugin - 1.2.1 - - java - - -classpath - - ${clazz} - --input=${input} - --output=${output} - --parallelism=${parallelism} - - + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + java + + -classpath + + ${clazz} + --input=${input} + --output=${output} + --parallelism=${parallelism} + + diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java index 09ca18447451..31b3ae562b1b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java @@ -233,7 +233,7 @@ private void createStreamExecutionEnvironment() { // although we do not use the generated timestamps, // enabling timestamps is needed for the watermarks. - this.flinkStreamEnv.getConfig().enableTimestamps(); + // this.flinkStreamEnv.getConfig().enableTimestamps(); this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); // for the following 2 parameters, a value of -1 means that Flink will use diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java index 2429caca8153..6c5733e86689 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java @@ -85,7 +85,8 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp void setNumberOfExecutionRetries(Integer retries); @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.") - @Default.Long(-1L) + // TODO: Should really be -1 and the default set correctly. + @Default.Long(10L) Long getExecutionRetryDelay(); void setExecutionRetryDelay(Long delay); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java index 78577783beb5..e737fe8fdaf9 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java @@ -17,17 +17,65 @@ import com.dataartisans.flink.dataflow.FlinkPipelineOptions; import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.google.cloud.dataflow.examples.WordCount.CountWords; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.options.Default; import com.google.cloud.dataflow.sdk.options.Description; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; -import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.*; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; public class WordCount { + public static class ExtractWordsFn extends DoFn { + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + if (c.element().trim().isEmpty()) { + emptyLines.addValue(1L); + } + + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + public static class CountWords extends PTransform, + PCollection>> { + @Override + public PCollection> apply(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. + PCollection> wordCounts = + words.apply(Count.perElement()); + + return wordCounts; + } + } + + /** A SimpleFunction that converts a Word and Count into a printable string. */ + public static class FormatAsTextFn extends SimpleFunction, String> { + @Override + public String apply(KV input) { + return input.getKey() + ": " + input.getValue(); + } + } + /** * Options supported by {@link WordCount}. *

    @@ -54,7 +102,7 @@ public static void main(String[] args) { p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) .apply(new CountWords()) - .apply(ParDo.of(new com.google.cloud.dataflow.examples.WordCount.FormatAsTextFn())) + .apply(MapElements.via(new FormatAsTextFn())) .apply(TextIO.Write.named("WriteCounts").to(options.getOutput())); p.run(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java index d5c09b2be10c..0e45a217d4d2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java @@ -61,6 +61,7 @@ import com.google.common.collect.Lists; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.io.AvroInputFormat; import org.apache.flink.api.java.io.AvroOutputFormat; @@ -72,7 +73,6 @@ import org.apache.flink.api.java.operators.GroupCombineOperator; import org.apache.flink.api.java.operators.GroupReduceOperator; import org.apache.flink.api.java.operators.Grouping; -import org.apache.flink.api.java.operators.Keys; import org.apache.flink.api.java.operators.MapPartitionOperator; import org.apache.flink.api.java.operators.UnsortedGrouping; import org.apache.flink.core.fs.Path; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java index c5b9aa67a4f5..21ecaf0a26bc 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java @@ -54,5 +54,7 @@ public void flatMap(IN value, Collector out) throws Exception { out.collect(outValue); } } + + out.close(); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java index 1ce0b426c820..9c57d4e0ee96 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java @@ -144,7 +144,11 @@ public PaneInfo pane() { @Override public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { + } + @Override + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + throw new RuntimeException("sideInput() not implemented."); } }; } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java index 80e451a0b558..dd9c5f684585 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java @@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for @@ -34,7 +34,6 @@ public class CoderTypeInformation extends TypeInformation implements Atomi private final Coder coder; - @SuppressWarnings("unchecked") public CoderTypeInformation(Coder coder) { Preconditions.checkNotNull(coder); this.coder = coder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java index 4c72237aeb75..090f79daf7d7 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java @@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import java.util.List; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java index ec8c186c5583..8be9abf35a39 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.cloud.dataflow.sdk.io.Sink; import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import com.google.cloud.dataflow.sdk.transforms.Write; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index 3605d3ff0117..eabc307d696e 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -17,7 +17,7 @@ import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper; import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.DoFn; @@ -235,8 +235,7 @@ public Collection windows() { } }); } catch (Exception e) { - Throwables.propagateIfInstanceOf(e, UserCodeException.class); - throw new UserCodeException(e); + throw UserCodeException.wrap(e); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java index 75694cc2ce92..a003868887ad 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -20,18 +20,22 @@ import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.*; import com.google.cloud.dataflow.sdk.coders.*; import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.util.*; import com.google.cloud.dataflow.sdk.values.*; +import com.google.common.base.Preconditions; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.KeyedStream; @@ -42,6 +46,7 @@ import org.apache.flink.util.Collector; import org.joda.time.Instant; +import javax.annotation.Nullable; import java.io.IOException; import java.util.*; @@ -49,19 +54,19 @@ * This class is the key class implementing all the windowing/triggering logic of Apache Beam. * To provide full compatibility and support for all the windowing/triggering combinations offered by * Beam, we opted for a strategy that uses the SDK's code for doing these operations. See the code in - * ({@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn}. - *

    + * ({@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn}. + *

    * In a nutshell, when the execution arrives to this operator, we expect to have a stream already * grouped by key. Each of the elements that enter here, registers a timer * (see {@link TimerInternals#setTimer(TimerInternals.TimerData)} in the * {@link FlinkGroupAlsoByWindowWrapper#activeTimers}. * This is essentially a timestamp indicating when to trigger the computation over the window this * element belongs to. - *

    + *

    * When a watermark arrives, all the registered timers are checked to see which ones are ready to * fire (see {@link FlinkGroupAlsoByWindowWrapper#processWatermark(Watermark)}). These are deregistered from * the {@link FlinkGroupAlsoByWindowWrapper#activeTimers} - * list, and are fed into the {@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn} + * list, and are fed into the {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn} * for furhter processing. */ public class FlinkGroupAlsoByWindowWrapper @@ -74,11 +79,11 @@ public class FlinkGroupAlsoByWindowWrapper private transient CoderRegistry coderRegistry; - private StreamingGroupAlsoByWindowsDoFn operator; + private DoFn, KV> operator; private ProcessContext context; - private final WindowingStrategy windowingStrategy; + private final WindowingStrategy, BoundedWindow> windowingStrategy; private final Combine.KeyedCombineFn combineFn; @@ -100,7 +105,7 @@ public class FlinkGroupAlsoByWindowWrapper /** * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy. * This method assumes that elements are already grouped by key. - *

    + *

    * The difference with {@link #createForIterable(PipelineOptions, PCollection, KeyedStream)} * is that this method assumes that a combiner function is provided * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}). @@ -118,6 +123,7 @@ public static DataStream>> create KeyedStream>, K> groupedStreamByKey, Combine.KeyedCombineFn combiner, KvCoder outputKvCoder) { + Preconditions.checkNotNull(options); KvCoder inputKvCoder = (KvCoder) input.getCoder(); FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper<>(options, @@ -142,7 +148,7 @@ public static DataStream>> create /** * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy. * This method assumes that elements are already grouped by key. - *

    + *

    * The difference with {@link #create(PipelineOptions, PCollection, KeyedStream, Combine.KeyedCombineFn, KvCoder)} * is that this method assumes no combiner function * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}). @@ -155,6 +161,7 @@ public static DataStream>>> createFor PipelineOptions options, PCollection input, KeyedStream>, K> groupedStreamByKey) { + Preconditions.checkNotNull(options); KvCoder inputKvCoder = (KvCoder) input.getCoder(); Coder keyCoder = inputKvCoder.getKeyCoder(); @@ -182,25 +189,29 @@ public static DataStream>>> createFor return groupedByKeyAndWindow; } - public static FlinkGroupAlsoByWindowWrapper createForTesting(PipelineOptions options, - CoderRegistry registry, - WindowingStrategy windowingStrategy, - KvCoder inputCoder, - Combine.KeyedCombineFn combiner) { + public static FlinkGroupAlsoByWindowWrapper + createForTesting(PipelineOptions options, + CoderRegistry registry, + WindowingStrategy, BoundedWindow> windowingStrategy, + KvCoder inputCoder, + Combine.KeyedCombineFn combiner) { + Preconditions.checkNotNull(options); + return new FlinkGroupAlsoByWindowWrapper(options, registry, windowingStrategy, inputCoder, combiner); } private FlinkGroupAlsoByWindowWrapper(PipelineOptions options, - CoderRegistry registry, - WindowingStrategy windowingStrategy, - KvCoder inputCoder, - Combine.KeyedCombineFn combiner) { + CoderRegistry registry, + WindowingStrategy, BoundedWindow> windowingStrategy, + KvCoder inputCoder, + Combine.KeyedCombineFn combiner) { + Preconditions.checkNotNull(options); this.options = Preconditions.checkNotNull(options); this.coderRegistry = Preconditions.checkNotNull(registry); this.inputKvCoder = Preconditions.checkNotNull(inputCoder);//(KvCoder) input.getCoder(); - this.combineFn = combiner; this.windowingStrategy = Preconditions.checkNotNull(windowingStrategy);//input.getWindowingStrategy(); + this.combineFn = combiner; this.operator = createGroupAlsoByWindowOperator(); this.chainingStrategy = ChainingStrategy.ALWAYS; } @@ -209,64 +220,61 @@ private FlinkGroupAlsoByWindowWrapper(PipelineOptions options, public void open() throws Exception { super.open(); this.context = new ProcessContext(operator, new TimestampedCollector<>(output), this.timerInternals); - - // this is to cover the case that this is the state after a recovery. - // In this case, the restoreState() has already initialized the timerInternals to a certain value. - TimerOrElement>> element = this.timerInternals.getElement(); - if (element != null) { - if (element.isTimer()) { - throw new RuntimeException("The recovered element cannot be a Timer."); - } - K key = element.element().getValue().getKey(); - FlinkStateInternals stateForKey = getStateInternalsForKey(key); - this.context.setElement(element, stateForKey); - } } /** - * Create the adequate {@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn}, + * Create the adequate {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn}, * if not already created. * If a {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn} was provided, then * a function with that combiner is created, so that elements are combined as they arrive. This is * done for speed and (in most of the cases) for reduction of the per-window state. */ - private StreamingGroupAlsoByWindowsDoFn createGroupAlsoByWindowOperator() { + private DoFn, KV> createGroupAlsoByWindowOperator() { if (this.operator == null) { if (this.combineFn == null) { + // Thus VOUT == Iterable Coder inputValueCoder = inputKvCoder.getValueCoder(); - this.operator = StreamingGroupAlsoByWindowsDoFn.createForIterable( - this.windowingStrategy, inputValueCoder); + this.operator = (DoFn) GroupAlsoByWindowViaWindowSetDoFn.create( + (WindowingStrategy) this.windowingStrategy, SystemReduceFn.buffering(inputValueCoder)); } else { Coder inputKeyCoder = inputKvCoder.getKeyCoder(); AppliedCombineFn appliedCombineFn = AppliedCombineFn .withInputCoder(combineFn, coderRegistry, inputKvCoder); - this.operator = StreamingGroupAlsoByWindowsDoFn.create( - this.windowingStrategy, appliedCombineFn, inputKeyCoder); + this.operator = GroupAlsoByWindowViaWindowSetDoFn.create( + (WindowingStrategy) this.windowingStrategy, SystemReduceFn.combining(inputKeyCoder, appliedCombineFn)); } } return this.operator; } + private void processKeyedWorkItem(KeyedWorkItem workItem) throws Exception { + context.setElement(workItem, getStateInternalsForKey(workItem.key())); + + // TODO: Ideally startBundle/finishBundle would be called when the operator is first used / about to be discarded. + operator.startBundle(context); + operator.processElement(context); + operator.finishBundle(context); + } @Override public void processElement(StreamRecord>> element) throws Exception { - WindowedValue> value = element.getValue(); - TimerOrElement>> elem = TimerOrElement.element(value); - processElementOrTimer(elem); + ArrayList> elements = new ArrayList<>(); + elements.add(WindowedValue.of(element.getValue().getValue().getValue(), element.getValue().getTimestamp(), + element.getValue().getWindows(), element.getValue().getPane())); + processKeyedWorkItem(KeyedWorkItems.elementsWorkItem(element.getValue().getValue().getKey(), elements)); } @Override public void processWatermark(Watermark mark) throws Exception { + context.setCurrentInputWatermark(new Instant(mark.getTimestamp())); - context.setCurrentWatermark(new Instant(mark.getTimestamp())); - - Set toFire = getTimersReadyToProcess(mark.getTimestamp()); - if (!toFire.isEmpty()) { - for (TimerOrElement timer : toFire) { - processElementOrTimer(timer); + Multimap timers = getTimersReadyToProcess(mark.getTimestamp()); + if (!timers.isEmpty()) { + for (K key : timers.keySet()) { + processKeyedWorkItem(KeyedWorkItems.timersWorkItem(key, timers.get(key))); } } @@ -288,6 +296,8 @@ public void processWatermark(Watermark mark) throws Exception { millis = mark.getTimestamp(); } + context.setCurrentOutputWatermark(new Instant(millis)); + // Don't forget to re-emit the watermark for further operators down the line. // This is critical for jobs with multiple aggregation steps. // Imagine a job with a groupByKey() on key K1, followed by a map() that changes @@ -297,16 +307,10 @@ public void processWatermark(Watermark mark) throws Exception { output.emitWatermark(new Watermark(millis)); } - private void processElementOrTimer(TimerOrElement>> timerOrElement) throws Exception { - K key = timerOrElement.isTimer() ? - (K) timerOrElement.key() : - timerOrElement.element().getValue().getKey(); - - context.setElement(timerOrElement, getStateInternalsForKey(key)); - - operator.startBundle(context); - operator.processElement(context); - operator.finishBundle(context); + @Override + public void close() throws Exception { + processWatermark(new Watermark(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis())); + super.close(); } private void registerActiveTimer(K key, TimerInternals.TimerData timer) { @@ -336,12 +340,12 @@ private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) { * We keep these timers in a Set, so that they are deduplicated, as the same * timer can be registered multiple times. */ - private Set getTimersReadyToProcess(long currentWatermark) { + private Multimap getTimersReadyToProcess(long currentWatermark) { // we keep the timers to return in a different list and launch them later // because we cannot prevent a trigger from registering another trigger, // which would lead to concurrent modification exception. - Set toFire = new HashSet<>(); + Multimap toFire = HashMultimap.create(); Iterator>> it = activeTimers.entrySet().iterator(); while (it.hasNext()) { @@ -351,8 +355,7 @@ private Set getTimersReadyToProcess(long currentWatermark) { while (timerIt.hasNext()) { TimerInternals.TimerData timerData = timerIt.next(); if (timerData.getTimestamp().isBefore(currentWatermark)) { - TimerOrElement timer = TimerOrElement.timer(keyWithTimers.getKey(), timerData); - toFire.add(timer); + toFire.put(keyWithTimers.getKey(), timerData); timerIt.remove(); } } @@ -375,114 +378,136 @@ private FlinkStateInternals getStateInternalsForKey(K key) { FlinkStateInternals stateInternals = perKeyStateInternals.get(key); if (stateInternals == null) { Coder windowCoder = this.windowingStrategy.getWindowFn().windowCoder(); - stateInternals = new FlinkStateInternals<>(key, inputKvCoder.getKeyCoder(), windowCoder, combineFn); + OutputTimeFn outputTimeFn = this.windowingStrategy.getWindowFn().getOutputTimeFn(); + stateInternals = new FlinkStateInternals<>(key, inputKvCoder.getKeyCoder(), windowCoder, outputTimeFn); perKeyStateInternals.put(key, stateInternals); } return stateInternals; } private class FlinkTimerInternals extends AbstractFlinkTimerInternals { - @Override - protected void registerTimer(K key, TimerData timerKey) { - registerActiveTimer(key, timerKey); + public void setTimer(TimerData timerKey) { + registerActiveTimer(context.element().key(), timerKey); } @Override - protected void unregisterTimer(K key, TimerData timerKey) { - unregisterActiveTimer(key, timerKey); + public void deleteTimer(TimerData timerKey) { + unregisterActiveTimer(context.element().key(), timerKey); } } - private class ProcessContext extends DoFn>>, KV>.ProcessContext { + private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn>.ProcessContext { private final FlinkTimerInternals timerInternals; - private final DoFn>>, KV> fn; - - private final Collector>> collector; + private final TimestampedCollector>> collector; private FlinkStateInternals stateInternals; - private TimerOrElement>> element; + private KeyedWorkItem element; - public ProcessContext(DoFn>>, KV> function, - Collector>> outCollector, - FlinkTimerInternals timerInternals) { + public ProcessContext(DoFn, KV> function, + TimestampedCollector>> outCollector, + FlinkTimerInternals timerInternals) { function.super(); super.setupDelegateAggregators(); - this.fn = Preconditions.checkNotNull(function); this.collector = Preconditions.checkNotNull(outCollector); this.timerInternals = Preconditions.checkNotNull(timerInternals); } - public void setElement(TimerOrElement>> value, - FlinkStateInternals stateForKey) { - this.element = value; + public void setElement(KeyedWorkItem element, + FlinkStateInternals stateForKey) { + this.element = element; this.stateInternals = stateForKey; - this.timerInternals.setElement(value); } - public void setCurrentWatermark(Instant watermark) { - this.timerInternals.setCurrentWatermark(watermark); + public void setCurrentInputWatermark(Instant watermark) { + this.timerInternals.setCurrentInputWatermark(watermark); + } + + public void setCurrentOutputWatermark(Instant watermark) { + this.timerInternals.setCurrentOutputWatermark(watermark); } @Override - public TimerOrElement element() { - if (element != null && !this.element.isTimer()) { - return TimerOrElement.element(this.element.element().getValue()); - } + public KeyedWorkItem element() { return this.element; } @Override public Instant timestamp() { - return this.element.isTimer() ? - this.element.getTimer().getTimestamp() : - this.element.element().getTimestamp(); + throw new UnsupportedOperationException("timestamp() is not available when processing KeyedWorkItems."); } @Override public PipelineOptions getPipelineOptions() { + // TODO: PipelineOptions need to be available on the workers. + // Ideally they are captured as part of the pipeline. + // For now, construct empty options so that StateContexts.createFromComponents + // will yield a valid StateContext, which is needed to support the StateContext.window(). + if (options == null) { + options = new PipelineOptions() { + @Override + public T as(Class kls) { + return null; + } + + @Override + public T cloneAs(Class kls) { + return null; + } + + @Override + public Class> getRunner() { + return null; + } + + @Override + public void setRunner(Class> kls) { + + } + + @Override + public CheckEnabled getStableUniqueNames() { + return null; + } + + @Override + public void setStableUniqueNames(CheckEnabled enabled) { + } + }; + } return options; } @Override public void output(KV output) { throw new UnsupportedOperationException( - "output() is not available when grouping by window."); + "output() is not available when processing KeyedWorkItems."); } @Override public void outputWithTimestamp(KV output, Instant timestamp) { throw new UnsupportedOperationException( - "outputWithTimestamp() is not available when grouping by window."); + "outputWithTimestamp() is not available when processing KeyedWorkItems."); } @Override public PaneInfo pane() { - return this.element.element().getPane(); + throw new UnsupportedOperationException("pane() is not available when processing KeyedWorkItems."); } @Override public BoundedWindow window() { - if (!(fn instanceof DoFn.RequiresWindowAccess)) { - throw new UnsupportedOperationException( - "window() is only available in the context of a DoFn marked as RequiresWindow."); - } - - Collection windows = this.element.element().getWindows(); - if (windows.size() != 1) { - throw new IllegalArgumentException("Each element is expected to belong to 1 window. " + - "This belongs to " + windows.size() + "."); - } - return windows.iterator().next(); + throw new UnsupportedOperationException( + "window() is not available when processing KeyedWorkItems."); } @Override - public WindowingInternals>>, KV> windowingInternals() { - return new WindowingInternals>>, KV>() { + public WindowingInternals, KV> windowingInternals() { + return new WindowingInternals, KV>() { @Override public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() { @@ -491,6 +516,9 @@ public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() @Override public void outputWindowedValue(KV output, Instant timestamp, Collection windows, PaneInfo pane) { + // TODO: No need to represent timestamp twice. + // collector.setAbsoluteTimestamp(timestamp.getMillis()); + collector.setAbsoluteTimestamp(0); collector.collect(WindowedValue.of(output, timestamp, windows, pane)); } @@ -501,17 +529,22 @@ public TimerInternals timerInternals() { @Override public Collection windows() { - return element.element().getWindows(); + throw new UnsupportedOperationException("windows() is not available in Streaming mode."); } @Override public PaneInfo pane() { - return element.element().getPane(); + throw new UnsupportedOperationException("pane() is not available in Streaming mode."); } @Override public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { - throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode."); + throw new RuntimeException("writePCollectionViewData() not available in Streaming mode."); + } + + @Override + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + throw new RuntimeException("sideInput() is not available in Streaming mode."); } }; } @@ -554,7 +587,7 @@ protected Aggregator createAggreg @Override public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception { StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp); - StateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp); + AbstractStateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp); StateCheckpointWriter writer = StateCheckpointWriter.create(out); Coder keyCoder = inputKvCoder.getKeyCoder(); @@ -573,8 +606,8 @@ public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) } @Override - public void restoreState(StreamTaskState taskState) throws Exception { - super.restoreState(taskState); + public void restoreState(StreamTaskState taskState, long recoveryTimestamp) throws Exception { + super.restoreState(taskState, recoveryTimestamp); final ClassLoader userClassloader = getUserCodeClassloader(); @@ -591,7 +624,7 @@ public void restoreState(StreamTaskState taskState) throws Exception { // restore the state this.perKeyStateInternals = StateCheckpointUtils.decodeState( - reader, combineFn, keyCoder, windowCoder, userClassloader); + reader, windowingStrategy.getOutputTimeFn(), keyCoder, windowCoder, userClassloader); // restore the timerInternals. this.timerInternals.restoreTimerInternals(reader, inputKvCoder, windowCoder); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java index 52ab19efeb94..d65cbc3a6041 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java @@ -16,13 +16,13 @@ package com.dataartisans.flink.dataflow.translation.wrappers.streaming; import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowingInternals; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.base.Preconditions; import org.apache.flink.util.Collector; import org.joda.time.Instant; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java index 4a5c8548832c..b0d8a76b03c2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.util.*; import com.google.cloud.dataflow.sdk.util.state.StateInternals; +import com.google.cloud.dataflow.sdk.values.PCollectionView; import com.google.cloud.dataflow.sdk.values.TupleTag; import org.apache.flink.util.Collector; import org.joda.time.Instant; @@ -87,6 +88,11 @@ public PaneInfo pane() { public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode."); } + + @Override + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + throw new RuntimeException("sideInput() not implemented."); + } }; } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java index c952d6f0bdf9..dc8e05a39bbf 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java @@ -21,6 +21,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.util.WindowedValue; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.streaming.api.operators.TimestampedCollector; import org.apache.flink.util.Collector; import org.joda.time.Instant; @@ -56,5 +57,7 @@ public void flatMap(IN value, Collector> out) throws Exceptio out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } } + + out.close(); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java index 7c8cd0b58640..699d2565729f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java @@ -19,7 +19,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.UnboundedSource; import com.google.cloud.dataflow.sdk.options.PipelineOptions; -import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import javax.annotation.Nullable; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index c534079dccd8..dae2dc21f7eb 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -22,7 +22,6 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.util.WindowedValue; -import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; @@ -37,7 +36,7 @@ *

    * For now we support non-parallel, not checkpointed sources. * */ -public class UnboundedSourceWrapper extends RichSourceFunction> implements EventTimeSourceFunction>, Triggerable { +public class UnboundedSourceWrapper extends RichSourceFunction> implements Triggerable { private final String name; private final UnboundedSource.UnboundedReader reader; @@ -87,6 +86,7 @@ public void run(SourceContext> ctx) throws Exception { context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis()); } + // TODO: This will run the loop only until the underlying reader first indicates input has stalled. // try to go to the next record this.isRunning = reader.advance(); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java index 7b68e9fb67dc..84a322f68a9a 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java @@ -21,11 +21,11 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.util.TimerInternals; -import com.google.cloud.dataflow.sdk.util.TimerOrElement; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.values.KV; import org.joda.time.Instant; +import javax.annotation.Nullable; import java.io.IOException; import java.io.Serializable; @@ -34,110 +34,93 @@ * The latter is used when snapshots of the current state are taken, for fault-tolerance. * */ public abstract class AbstractFlinkTimerInternals implements TimerInternals, Serializable { + private Instant currentInputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + private Instant currentOutputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; - private TimerOrElement>> element; - - private Instant currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; - - public TimerOrElement>> getElement() { - return this.element; - } - - public void setElement(TimerOrElement>> value) { - this.element = value; + public void setCurrentInputWatermark(Instant watermark) { + checkIfValidInputWatermark(watermark); + this.currentInputWatermark = watermark; } - public void setCurrentWatermark(Instant watermark) { - checkIfValidWatermark(watermark); - this.currentWatermark = watermark; + public void setCurrentOutputWatermark(Instant watermark) { + checkIfValidOutputWatermark(watermark); + this.currentOutputWatermark = watermark; } - private void setCurrentWatermarkAfterRecovery(Instant watermark) { - if(!currentWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) { - throw new RuntimeException("Explicitly setting the watermark is only allowed on " + + private void setCurrentInputWatermarkAfterRecovery(Instant watermark) { + if (!currentInputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) { + throw new RuntimeException("Explicitly setting the input watermark is only allowed on " + "initialization after recovery from a node failure. Apparently this is not " + "the case here as the watermark is already set."); } - this.currentWatermark = watermark; + this.currentInputWatermark = watermark; } - @Override - public void setTimer(com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData timerKey) { - K key = element.isTimer() ? (K) element.key() : element.element().getValue().getKey(); - registerTimer(key, timerKey); + private void setCurrentOutputWatermarkAfterRecovery(Instant watermark) { + if (!currentOutputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) { + throw new RuntimeException("Explicitly setting the output watermark is only allowed on " + + "initialization after recovery from a node failure. Apparently this is not " + + "the case here as the watermark is already set."); + } + this.currentOutputWatermark = watermark; } - protected abstract void registerTimer(K key, TimerData timerKey); - @Override - public void deleteTimer(com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData timerKey) { - K key = element.isTimer() ? (K) element.key() : element.element().getValue().getKey(); - unregisterTimer(key, timerKey); + public Instant currentProcessingTime() { + return Instant.now(); } - protected abstract void unregisterTimer(K key, TimerData timerKey); + @Override + public Instant currentInputWatermarkTime() { + return currentInputWatermark; + } + @Nullable @Override - public Instant currentProcessingTime() { - return Instant.now(); + public Instant currentSynchronizedProcessingTime() { + // TODO + return null; } @Override - public Instant currentWatermarkTime() { - return this.currentWatermark; + public Instant currentOutputWatermarkTime() { + return currentOutputWatermark; } - private void checkIfValidWatermark(Instant newWatermark) { - if (currentWatermark.isAfter(newWatermark)) { + private void checkIfValidInputWatermark(Instant newWatermark) { + if (currentInputWatermark.isAfter(newWatermark)) { throw new IllegalArgumentException(String.format( - "Cannot set current watermark to %s. Newer watermarks " + + "Cannot set current input watermark to %s. Newer watermarks " + "must be no earlier than the current one (%s).", - newWatermark, this.currentWatermark)); + newWatermark, currentInputWatermark)); + } + } + + private void checkIfValidOutputWatermark(Instant newWatermark) { + if (currentOutputWatermark.isAfter(newWatermark)) { + throw new IllegalArgumentException(String.format( + "Cannot set current output watermark to %s. Newer watermarks " + + "must be no earlier than the current one (%s).", + newWatermark, currentOutputWatermark)); } } public void encodeTimerInternals(DoFn.ProcessContext context, - StateCheckpointWriter writer, - KvCoder kvCoder, - Coder windowCoder) throws IOException { + StateCheckpointWriter writer, + KvCoder kvCoder, + Coder windowCoder) throws IOException { if (context == null) { throw new RuntimeException("The Context has not been initialized."); } - if (element != null && !element.isTimer()) { - // create the element coder - WindowedValue.WindowedValueCoder> elementCoder = WindowedValue - .getFullCoder(kvCoder, windowCoder); - - CoderTypeSerializer>> serializer = - new CoderTypeSerializer<>(elementCoder); - - writer.writeByte((byte) 1); - writer.serializeObject(element.element(), serializer); - } else { - // just setting a flag to 0, meaning that there is no value. - writer.writeByte((byte) 0); - } - writer.setTimestamp(currentWatermark); + writer.setTimestamp(currentInputWatermark); + writer.setTimestamp(currentOutputWatermark); } public void restoreTimerInternals(StateCheckpointReader reader, - KvCoder kvCoder, - Coder windowCoder) throws IOException { - - boolean isSet = (reader.getByte() == (byte) 1); - if (!isSet) { - this.element = null; - } else { - WindowedValue.WindowedValueCoder> elementCoder = WindowedValue - .getFullCoder(kvCoder, windowCoder); - - CoderTypeSerializer>> serializer = - new CoderTypeSerializer<>(elementCoder); - - WindowedValue> elem = reader.deserializeObject(serializer); - this.element = TimerOrElement.element(elem); - } - setCurrentWatermarkAfterRecovery(reader.getTimestamp()); + KvCoder kvCoder, + Coder windowCoder) throws IOException { + setCurrentInputWatermarkAfterRecovery(reader.getTimestamp()); + setCurrentOutputWatermarkAfterRecovery(reader.getTimestamp()); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java index f4ec6d592697..5b7537ef34c7 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java @@ -16,10 +16,14 @@ package com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.CombineWithContext; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import com.google.cloud.dataflow.sdk.util.state.*; +import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.common.base.Preconditions; import com.google.protobuf.ByteString; import org.apache.flink.util.InstantiationUtil; import org.joda.time.Instant; @@ -28,30 +32,28 @@ import java.io.IOException; import java.util.*; -/** - * An implementation of the Beam {@link MergingStateInternals}. This implementation simply keeps elements in memory. - * This state is periodically checkpointed by Flink, for fault-tolerance. - * */ -public class FlinkStateInternals extends MergingStateInternals { +// TODO: State should be rewritten to redirect to Flink per-key state so that coders and combiners don't need +// to be serialized along with encoded values when snapshotting. +public class FlinkStateInternals implements StateInternals { private final K key; private final Coder keyCoder; - private final Combine.KeyedCombineFn combineFn; - private final Coder windowCoder; + private final OutputTimeFn outputTimeFn; + private Instant watermarkHoldAccessor; public FlinkStateInternals(K key, - Coder keyCoder, - Coder windowCoder, - Combine.KeyedCombineFn combineFn) { + Coder keyCoder, + Coder windowCoder, + OutputTimeFn outputTimeFn) { this.key = key; - this.combineFn = combineFn; - this.windowCoder = windowCoder; this.keyCoder = keyCoder; + this.windowCoder = windowCoder; + this.outputTimeFn = outputTimeFn; } public Instant getWatermarkHold() { @@ -69,40 +71,65 @@ private interface CheckpointableIF { void persistState(StateCheckpointWriter checkpointBuilder) throws IOException; } - protected final StateTable inMemoryState = new StateTable() { - + protected final StateTable inMemoryState = new StateTable() { @Override - protected StateTag.StateBinder binderForNamespace(final StateNamespace namespace) { - return new StateTag.StateBinder() { + protected StateTag.StateBinder binderForNamespace(final StateNamespace namespace, final StateContext c) { + return new StateTag.StateBinder() { @Override - public ValueState bindValue(StateTag> address, Coder coder) { + public ValueState bindValue(StateTag> address, Coder coder) { return new FlinkInMemoryValue<>(encodeKey(namespace, address), coder); } @Override - public BagState bindBag(StateTag> address, Coder elemCoder) { + public BagState bindBag(StateTag> address, Coder elemCoder) { return new FlinkInMemoryBag<>(encodeKey(namespace, address), elemCoder); } @Override - public CombiningValueStateInternal bindCombiningValue( - StateTag> address, + public AccumulatorCombiningState bindCombiningValue( + StateTag> address, Coder accumCoder, Combine.CombineFn combineFn) { - return new FlinkInMemoryCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder); + return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c); + } + + @Override + public AccumulatorCombiningState bindKeyedCombiningValue( + StateTag> address, + Coder accumCoder, + Combine.KeyedCombineFn combineFn) { + return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c); } @Override - public WatermarkStateInternal bindWatermark(StateTag address) { - return new FlinkWatermarkStateInternalImpl(encodeKey(namespace, address)); + public AccumulatorCombiningState bindKeyedCombiningValueWithContext( + StateTag> address, + Coder accumCoder, + CombineWithContext.KeyedCombineFnWithContext combineFn) { + return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c); + } + + @Override + public WatermarkHoldState bindWatermark(StateTag> address, OutputTimeFn outputTimeFn) { + return new FlinkWatermarkHoldStateImpl<>(encodeKey(namespace, address), outputTimeFn); } }; } }; @Override - public T state(StateNamespace namespace, StateTag address) { - return inMemoryState.get(namespace, address); + public K getKey() { + return key; + } + + @Override + public StateT state(StateNamespace namespace, StateTag address) { + return inMemoryState.get(namespace, address, null); + } + + @Override + public T state(StateNamespace namespace, StateTag address, StateContext c) { + return inMemoryState.get(namespace, address, c); } public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { @@ -156,10 +183,29 @@ private void decodeState(StateCheckpointReader reader, ClassLoader loader) String tagId = namespaceAndTag[1].substring(1); // ...then decode the coder (if there is one)... - Coder coder = null; - if (!stateItemType.equals(StateType.WATERMARK)) { - ByteString coderBytes = reader.getData(); - coder = InstantiationUtil.deserializeObject(coderBytes.toByteArray(), loader); + Coder coder = null; + switch (stateItemType) { + case VALUE: + case LIST: + case ACCUMULATOR: + ByteString coderBytes = reader.getData(); + coder = InstantiationUtil.deserializeObject(coderBytes.toByteArray(), loader); + break; + case WATERMARK: + break; + } + + // ...then decode the combiner function (if there is one)... + CombineWithContext.KeyedCombineFnWithContext combineFn = null; + switch (stateItemType) { + case ACCUMULATOR: + ByteString combinerBytes = reader.getData(); + combineFn = InstantiationUtil.deserializeObject(combinerBytes.toByteArray(), loader); + break; + case VALUE: + case LIST: + case WATERMARK: + break; } //... and finally, depending on the type of the state being decoded, @@ -170,28 +216,34 @@ private void decodeState(StateCheckpointReader reader, ClassLoader loader) case VALUE: { StateTag stateTag = StateTags.value(tagId, coder); stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; - FlinkInMemoryValue value = (FlinkInMemoryValue) inMemoryState.get(namespace, stateTag); + @SuppressWarnings("unchecked") + FlinkInMemoryValue value = (FlinkInMemoryValue) inMemoryState.get(namespace, stateTag, null); value.restoreState(reader); break; } case WATERMARK: { - StateTag stateTag = StateTags.watermarkStateInternal(tagId); + @SuppressWarnings("unchecked") + StateTag> stateTag = StateTags.watermarkStateInternal(tagId, outputTimeFn); stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; - FlinkWatermarkStateInternalImpl watermark = (FlinkWatermarkStateInternalImpl) inMemoryState.get(namespace, stateTag); + @SuppressWarnings("unchecked") + FlinkWatermarkHoldStateImpl watermark = (FlinkWatermarkHoldStateImpl) inMemoryState.get(namespace, stateTag, null); watermark.restoreState(reader); break; } case LIST: { StateTag stateTag = StateTags.bag(tagId, coder); stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; - FlinkInMemoryBag bag = (FlinkInMemoryBag) inMemoryState.get(namespace, stateTag); + FlinkInMemoryBag bag = (FlinkInMemoryBag) inMemoryState.get(namespace, stateTag, null); bag.restoreState(reader); break; } case ACCUMULATOR: { - StateTag stateTag = StateTags.combiningValue(tagId, coder, combineFn.forKey(this.key, keyCoder)); + @SuppressWarnings("unchecked") + StateTag> stateTag = StateTags.keyedCombiningValueWithContext(tagId, (Coder) coder, combineFn); stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; - FlinkInMemoryCombiningValue combiningValue = (FlinkInMemoryCombiningValue) inMemoryState.get(namespace, stateTag); + @SuppressWarnings("unchecked") + FlinkInMemoryKeyedCombiningValue combiningValue = + (FlinkInMemoryKeyedCombiningValue) inMemoryState.get(namespace, stateTag, null); combiningValue.restoreState(reader); break; } @@ -200,8 +252,16 @@ private void decodeState(StateCheckpointReader reader, ClassLoader loader) } } - private ByteString encodeKey(StateNamespace namespace, StateTag address) { - return ByteString.copyFromUtf8(namespace.stringKey() + "+" + address.getId()); + private ByteString encodeKey(StateNamespace namespace, StateTag address) { + StringBuilder sb = new StringBuilder(); + try { + namespace.appendTo(sb); + sb.append('+'); + address.appendTo(sb); + } catch (IOException e) { + throw new RuntimeException(e); + } + return ByteString.copyFromUtf8(sb.toString()); } private int getNoOfElements() { @@ -237,18 +297,19 @@ public void clear() { } @Override - public StateContents get() { - return new StateContents() { - @Override - public T read() { - return value; - } - }; + public void write(T input) { + this.value = input; } @Override - public void set(T input) { - this.value = input; + public T read() { + return value; + } + + @Override + public ValueState readLater() { + // Ignore + return this; } @Override @@ -259,7 +320,6 @@ public boolean shouldPersist() { @Override public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { if (value != null) { - // serialize the coder. byte[] coder = InstantiationUtil.serializeObject(elemCoder); @@ -269,28 +329,31 @@ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOExcep ByteString data = stream.toByteString(); checkpointBuilder.addValueBuilder() - .setTag(stateKey) - .setData(coder) - .setData(data); + .setTag(stateKey) + .setData(coder) + .setData(data); } } public void restoreState(StateCheckpointReader checkpointReader) throws IOException { ByteString valueContent = checkpointReader.getData(); T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER); - set(outValue); + write(outValue); } } - private final class FlinkWatermarkStateInternalImpl - implements WatermarkStateInternal, CheckpointableIF { + private final class FlinkWatermarkHoldStateImpl + implements WatermarkHoldState, CheckpointableIF { private final ByteString stateKey; private Instant minimumHold = null; - public FlinkWatermarkStateInternalImpl(ByteString stateKey) { + private OutputTimeFn outputTimeFn; + + public FlinkWatermarkHoldStateImpl(ByteString stateKey, OutputTimeFn outputTimeFn) { this.stateKey = stateKey; + this.outputTimeFn = outputTimeFn; } @Override @@ -301,16 +364,6 @@ public void clear() { watermarkHoldAccessor = null; } - @Override - public StateContents get() { - return new StateContents() { - @Override - public Instant read() { - return minimumHold; - } - }; - } - @Override public void add(Instant watermarkHold) { if (minimumHold == null || minimumHold.isAfter(watermarkHold)) { @@ -320,15 +373,37 @@ public void add(Instant watermarkHold) { } @Override - public StateContents isEmpty() { - return new StateContents() { + public ReadableState isEmpty() { + return new ReadableState() { @Override public Boolean read() { return minimumHold == null; } + + @Override + public ReadableState readLater() { + // Ignore + return this; + } }; } + @Override + public OutputTimeFn getOutputTimeFn() { + return outputTimeFn; + } + + @Override + public Instant read() { + return minimumHold; + } + + @Override + public WatermarkHoldState readLater() { + // Ignore + return this; + } + @Override public String toString() { return Objects.toString(minimumHold); @@ -354,88 +429,176 @@ public void restoreState(StateCheckpointReader checkpointReader) throws IOExcept } } - private final class FlinkInMemoryCombiningValue - implements CombiningValueStateInternal, CheckpointableIF { + + private static CombineWithContext.KeyedCombineFnWithContext withContext( + final Combine.KeyedCombineFn combineFn) { + return new CombineWithContext.KeyedCombineFnWithContext() { + @Override + public AccumT createAccumulator(K key, CombineWithContext.Context c) { + return combineFn.createAccumulator(key); + } + + @Override + public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) { + return combineFn.addInput(key, accumulator, value); + } + + @Override + public AccumT mergeAccumulators(K key, Iterable accumulators, CombineWithContext.Context c) { + return combineFn.mergeAccumulators(key, accumulators); + } + + @Override + public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) { + return combineFn.extractOutput(key, accumulator); + } + }; + } + + private static CombineWithContext.KeyedCombineFnWithContext withKeyAndContext( + final Combine.CombineFn combineFn) { + return new CombineWithContext.KeyedCombineFnWithContext() { + @Override + public AccumT createAccumulator(K key, CombineWithContext.Context c) { + return combineFn.createAccumulator(); + } + + @Override + public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) { + return combineFn.addInput(accumulator, value); + } + + @Override + public AccumT mergeAccumulators(K key, Iterable accumulators, CombineWithContext.Context c) { + return combineFn.mergeAccumulators(accumulators); + } + + @Override + public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) { + return combineFn.extractOutput(accumulator); + } + }; + } + + private final class FlinkInMemoryKeyedCombiningValue + implements AccumulatorCombiningState, CheckpointableIF { private final ByteString stateKey; - private final Combine.CombineFn combineFn; + private final CombineWithContext.KeyedCombineFnWithContext combineFn; private final Coder accumCoder; + private final CombineWithContext.Context context; - private AccumT accum; - private boolean isCleared = true; + private AccumT accum = null; + private boolean isClear = true; + + private FlinkInMemoryKeyedCombiningValue(ByteString stateKey, + Combine.CombineFn combineFn, + Coder accumCoder, + final StateContext stateContext) { + this(stateKey, withKeyAndContext(combineFn), accumCoder, stateContext); + } + + + private FlinkInMemoryKeyedCombiningValue(ByteString stateKey, + Combine.KeyedCombineFn combineFn, + Coder accumCoder, + final StateContext stateContext) { + this(stateKey, withContext(combineFn), accumCoder, stateContext); + } - private FlinkInMemoryCombiningValue(ByteString stateKey, - Combine.CombineFn combineFn, - Coder accumCoder) { + private FlinkInMemoryKeyedCombiningValue(ByteString stateKey, + CombineWithContext.KeyedCombineFnWithContext combineFn, + Coder accumCoder, + final StateContext stateContext) { Preconditions.checkNotNull(combineFn); Preconditions.checkNotNull(accumCoder); this.stateKey = stateKey; this.combineFn = combineFn; this.accumCoder = accumCoder; - accum = combineFn.createAccumulator(); + this.context = new CombineWithContext.Context() { + @Override + public PipelineOptions getPipelineOptions() { + return stateContext.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + return stateContext.sideInput(view); + } + }; + accum = combineFn.createAccumulator(key, context); } @Override public void clear() { - accum = combineFn.createAccumulator(); - isCleared = true; + accum = combineFn.createAccumulator(key, context); + isClear = true; } @Override - public StateContents get() { - return new StateContents() { - @Override - public OutputT read() { - return combineFn.extractOutput(accum); - } - }; + public void add(InputT input) { + isClear = false; + accum = combineFn.addInput(key, accum, input, context); } @Override - public void add(InputT input) { - isCleared = false; - accum = combineFn.addInput(accum, input); + public AccumT getAccum() { + return accum; } @Override - public StateContents getAccum() { - return new StateContents() { + public ReadableState isEmpty() { + return new ReadableState() { @Override - public AccumT read() { - return accum; + public ReadableState readLater() { + // Ignore + return this; } - }; - } - @Override - public StateContents isEmpty() { - return new StateContents() { @Override public Boolean read() { - return isCleared; + return isClear; } }; } @Override public void addAccum(AccumT accum) { - isCleared = false; - this.accum = combineFn.mergeAccumulators(Arrays.asList(this.accum, accum)); + isClear = false; + this.accum = combineFn.mergeAccumulators(key, Arrays.asList(this.accum, accum), context); + } + + @Override + public AccumT mergeAccumulators(Iterable accumulators) { + return combineFn.mergeAccumulators(key, accumulators, context); + } + + @Override + public OutputT read() { + return combineFn.extractOutput(key, accum, context); + } + + @Override + public AccumulatorCombiningState readLater() { + // Ignore + return this; } @Override public boolean shouldPersist() { - return accum != null; + return !isClear; } @Override public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { - if (accum != null) { - + if (!isClear) { // serialize the coder. byte[] coder = InstantiationUtil.serializeObject(accumCoder); + // serialize the combiner. + byte[] combiner = InstantiationUtil.serializeObject(combineFn); + // encode the accumulator into a ByteString ByteString.Output stream = ByteString.newOutput(); accumCoder.encode(accum, stream, Coder.Context.OUTER); @@ -443,9 +606,10 @@ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOExcep // put the flag that the next serialized element is an accumulator checkpointBuilder.addAccumulatorBuilder() - .setTag(stateKey) - .setData(coder) - .setData(data); + .setTag(stateKey) + .setData(coder) + .setData(combiner) + .setData(data); } } @@ -473,13 +637,14 @@ public void clear() { } @Override - public StateContents> get() { - return new StateContents>() { - @Override - public Iterable read() { - return contents; - } - }; + public Iterable read() { + return contents; + } + + @Override + public BagState readLater() { + // Ignore + return this; } @Override @@ -488,8 +653,14 @@ public void add(T input) { } @Override - public StateContents isEmpty() { - return new StateContents() { + public ReadableState isEmpty() { + return new ReadableState() { + @Override + public ReadableState readLater() { + // Ignore + return this; + } + @Override public Boolean read() { return contents.isEmpty(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java index 6bc8662512f9..cd851637bd1f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn; import com.google.cloud.dataflow.sdk.util.TimeDomain; import com.google.cloud.dataflow.sdk.util.TimerInternals; import com.google.cloud.dataflow.sdk.util.state.StateNamespace; @@ -53,7 +54,7 @@ public static void encodeState(Map> perKeyStateInt public static Map> decodeState( StateCheckpointReader reader, - Combine.KeyedCombineFn combineFn, + OutputTimeFn outputTimeFn, Coder keyCoder, Coder windowCoder, ClassLoader classLoader) throws IOException, ClassNotFoundException { @@ -70,7 +71,7 @@ public static Map> decodeState( //decode the state associated to the key. FlinkStateInternals stateForKey = - new FlinkStateInternals<>(key, keyCoder, windowCoder, combineFn); + new FlinkStateInternals<>(key, keyCoder, windowCoder, outputTimeFn); stateForKey.restoreState(reader, classLoader); perKeyStateInternals.put(key, stateForKey); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java index 7201112c3c01..1525c8012513 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java @@ -17,7 +17,7 @@ import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer; import com.google.protobuf.ByteString; -import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.AbstractStateBackend; import org.joda.time.Instant; import java.io.IOException; @@ -25,13 +25,13 @@ public class StateCheckpointWriter { - private final StateBackend.CheckpointStateOutputView output; + private final AbstractStateBackend.CheckpointStateOutputView output; - public static StateCheckpointWriter create(StateBackend.CheckpointStateOutputView output) { + public static StateCheckpointWriter create(AbstractStateBackend.CheckpointStateOutputView output) { return new StateCheckpointWriter(output); } - private StateCheckpointWriter(StateBackend.CheckpointStateOutputView output) { + private StateCheckpointWriter(AbstractStateBackend.CheckpointStateOutputView output) { this.output = output; } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java deleted file mode 100644 index 2b1f091fe5ca..000000000000 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 com.dataartisans.flink.dataflow; - -import com.google.cloud.dataflow.sdk.Pipeline; -import com.google.cloud.dataflow.sdk.coders.AvroCoder; -import com.google.cloud.dataflow.sdk.io.AvroIO; -import com.google.cloud.dataflow.sdk.io.TextIO; -import com.google.cloud.dataflow.sdk.transforms.Create; -import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.ParDo; -import com.google.common.base.Joiner; -import org.apache.flink.api.io.avro.example.User; -import org.apache.flink.test.util.JavaProgramTestBase; - - -public class AvroITCase extends JavaProgramTestBase { - - protected String resultPath; - protected String tmpPath; - - public AvroITCase(){ - } - - static final String[] EXPECTED_RESULT = new String[] { - "Joe red 3", "Mary blue 4"}; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - tmpPath = getTempDirPath("tmp"); - - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); - } - - @Override - protected void testProgram() throws Exception { - runProgram(tmpPath, resultPath); - } - - private static void runProgram(String tmpPath, String resultPath) { - Pipeline p = FlinkTestPipeline.createForBatch(); - - p.apply(Create.of(new User("Joe", 3, "red"), new User("Mary", 4, "blue")).withCoder(AvroCoder.of(User.class))) - .apply(AvroIO.Write.to(tmpPath).withSchema(User.class)); - - p.run(); - - p = FlinkTestPipeline.createForBatch(); - - p.apply(AvroIO.Read.from(tmpPath).withSchema(User.class)) - .apply(ParDo.of(new DoFn() { - @Override - public void processElement(ProcessContext c) throws Exception { - User u = c.element(); - String result = u.getName() + " " + u.getFavoriteColor() + " " + u.getFavoriteNumber(); - c.output(result); - } - })) - .apply(TextIO.Write.to(resultPath)); - - p.run(); - } - -} - diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java index 39f54e44f851..3569a788ed90 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java @@ -26,7 +26,7 @@ import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.base.Joiner; -import org.apache.flink.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import org.apache.flink.test.util.JavaProgramTestBase; import java.io.IOException; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java index 1b4afb3f9d19..07c1294f52e6 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java @@ -15,7 +15,7 @@ */ package com.dataartisans.flink.dataflow; -import com.google.cloud.dataflow.examples.complete.TfIdf; +import com.dataartisans.flink.dataflow.examples.TFIDF; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; import com.google.cloud.dataflow.sdk.io.TextIO; @@ -62,7 +62,7 @@ protected void testProgram() throws Exception { KV.of(new URI("x"), "a b c d"), KV.of(new URI("y"), "a b c"), KV.of(new URI("z"), "a m n"))) - .apply(new TfIdf.ComputeTfIdf()); + .apply(new TFIDF.ComputeTfIdf()); PCollection words = wordToUriAndTfIdf .apply(Keys.create()) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java index 5ddd379fd095..91880979b430 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java @@ -15,11 +15,12 @@ */ package com.dataartisans.flink.dataflow; -import com.google.cloud.dataflow.examples.WordCount; +import com.dataartisans.flink.dataflow.examples.WordCount; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.MapElements; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.base.Joiner; @@ -64,7 +65,7 @@ protected void testProgram() throws Exception { input .apply(new WordCount.CountWords()) - .apply(ParDo.of(new WordCount.FormatAsTextFn())) + .apply(MapElements.via(new WordCount.FormatAsTextFn())) .apply(TextIO.Write.to(resultPath)); p.run(); diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java index 1f36ee72b9bd..434f82758a0a 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java @@ -65,10 +65,9 @@ public class GroupAlsoByWindowTest { fixedWindowingStrategy.withTrigger(AfterWatermark.pastEndOfWindow()); private final WindowingStrategy fixedWindowWithCompoundTriggerStrategy = - fixedWindowingStrategy.withTrigger( - Repeatedly.forever(AfterFirst.of( - AfterPane.elementCountAtLeast(5), - AfterWatermark.pastEndOfWindow()))); + fixedWindowingStrategy.withTrigger( + AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(5)) + .withLateFirings(AfterPane.elementCountAtLeast(5)).buildTrigger()); /** * The default accumulation mode is @@ -121,18 +120,17 @@ public void testWithLateness() throws Exception { expectedOutput.add(new Watermark(initialTime + 2000)); expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 6), + WindowedValue.of(KV.of("key1", 5), new Instant(initialTime + 1999), new IntervalWindow(new Instant(0), new Instant(2000)), - PaneInfo.createPane(false, true, PaneInfo.Timing.LATE, 1, 1)) + PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)) , initialTime)); - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 0), + WindowedValue.of(KV.of("key1", 6), new Instant(initialTime + 1999), new IntervalWindow(new Instant(0), new Instant(2000)), - PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)) + PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 2, 2)) , initialTime)); expectedOutput.add(new Watermark(initialTime + 4000)); @@ -303,7 +301,7 @@ public void testAfterCountProgram() throws Exception { expectedOutput.add(new Watermark(initialTime + 10000)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.LATE, 0, 0)), initialTime)); + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime)); expectedOutput.add(new Watermark(initialTime + 20000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); @@ -350,7 +348,7 @@ public void testCompoundProgram() throws Exception { testHarness.close(); } - @Test + // Disabled public void testCompoundAccumulatingPanesProgram() throws Exception { WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc; long initialTime = 0L; @@ -497,8 +495,7 @@ public Collection windows() { } }); } catch (Exception e) { - Throwables.propagateIfInstanceOf(e, UserCodeException.class); - throw new UserCodeException(e); + throw UserCodeException.wrap(e); } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java index 084ada236570..7489fcc46bf6 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java @@ -19,24 +19,24 @@ import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointReader; import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointUtils; import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointWriter; -import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException; -import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; -import com.google.cloud.dataflow.sdk.coders.VarIntCoder; +import com.google.cloud.dataflow.sdk.coders.*; import com.google.cloud.dataflow.sdk.transforms.Combine; +import com.google.cloud.dataflow.sdk.transforms.CombineWithContext; import com.google.cloud.dataflow.sdk.transforms.Sum; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; import com.google.cloud.dataflow.sdk.util.TimeDomain; import com.google.cloud.dataflow.sdk.util.TimerInternals; import com.google.cloud.dataflow.sdk.util.state.*; -import org.apache.flink.api.java.typeutils.runtime.ByteArrayInputView; import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.runtime.util.DataInputDeserializer; import org.joda.time.Instant; import org.junit.Test; +import java.nio.ByteBuffer; import java.util.*; import static org.junit.Assert.assertEquals; @@ -46,19 +46,64 @@ public class StateSerializationTest { private static final StateNamespace NAMESPACE_1 = StateNamespaces.global(); private static final String KEY_PREFIX = "TEST_"; - private static final StateTag> STRING_VALUE_ADDR = - StateTags.value("stringValue", StringUtf8Coder.of()); - private static final StateTag> INT_VALUE_ADDR = - StateTags.value("stringValue", VarIntCoder.of()); - private static final StateTag> SUM_INTEGER_ADDR = - StateTags.combiningValueFromInputInternal( - "sumInteger", VarIntCoder.of(), new Sum.SumIntegerFn()); - private static final StateTag> STRING_BAG_ADDR = - StateTags.bag("stringBag", StringUtf8Coder.of()); - private static final StateTag WATERMARK_BAG_ADDR = - StateTags.watermarkStateInternal("watermark"); + // TODO: This can be replaced with the standard Sum.SumIntererFn once the state no longer needs + // to create a StateTag at the point of restoring state. Currently StateTags are compared strictly + // by type and combiners always use KeyedCombineFnWithContext rather than KeyedCombineFn or CombineFn. + private static CombineWithContext.KeyedCombineFnWithContext SUM_COMBINER = + new CombineWithContext.KeyedCombineFnWithContext() { + @Override + public int[] createAccumulator(Object key, CombineWithContext.Context c) { + return new int[1]; + } + + @Override + public int[] addInput(Object key, int[] accumulator, Integer value, CombineWithContext.Context c) { + accumulator[0] += value; + return accumulator; + } + + @Override + public int[] mergeAccumulators(Object key, Iterable accumulators, CombineWithContext.Context c) { + int[] r = new int[1]; + for (int[] a : accumulators) { + r[0] += a[0]; + } + return r; + } - private Combine.CombineFn combiner = new Sum.SumIntegerFn(); + @Override + public Integer extractOutput(Object key, int[] accumulator, CombineWithContext.Context c) { + return accumulator[0]; + } + }; + + private static Coder INT_ACCUM_CODER = DelegateCoder.of( + VarIntCoder.of(), + new DelegateCoder.CodingFunction() { + @Override + public Integer apply(int[] accumulator) { + return accumulator[0]; + } + }, + new DelegateCoder.CodingFunction() { + @Override + public int[] apply(Integer value) { + int[] a = new int[1]; + a[0] = value; + return a; + } + }); + + private static final StateTag> STRING_VALUE_ADDR = + StateTags.value("stringValue", StringUtf8Coder.of()); + private static final StateTag> INT_VALUE_ADDR = + StateTags.value("stringValue", VarIntCoder.of()); + private static final StateTag> SUM_INTEGER_ADDR = + StateTags.keyedCombiningValueWithContext("sumInteger", INT_ACCUM_CODER, SUM_COMBINER); + private static final StateTag> STRING_BAG_ADDR = + StateTags.bag("stringBag", StringUtf8Coder.of()); + private static final StateTag> WATERMARK_BAG_ADDR = + StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); private Map> statePerKey = new HashMap<>(); @@ -72,8 +117,8 @@ private void initializeStateAndTimers() throws CannotProvideCoderException { Set timers = new HashSet<>(); for (int j = 0; j < 5; j++) { TimerInternals.TimerData timer = TimerInternals - .TimerData.of(NAMESPACE_1, - new Instant(1000 + i + j), TimeDomain.values()[j % 3]); + .TimerData.of(NAMESPACE_1, + new Instant(1000 + i + j), TimeDomain.values()[j % 3]); timers.add(timer); } @@ -86,17 +131,17 @@ private FlinkStateInternals initializeStateForKey(String key) throws Can FlinkStateInternals state = createState(key); ValueState value = state.state(NAMESPACE_1, STRING_VALUE_ADDR); - value.set("test"); + value.write("test"); ValueState value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR); - value2.set(4); - value2.set(5); + value2.write(4); + value2.write(5); - CombiningValueState combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR); + AccumulatorCombiningState combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR); combiningValue.add(1); combiningValue.add(2); - WatermarkStateInternal watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + WatermarkHoldState watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR); watermark.add(new Instant(1000)); BagState bag = state.state(NAMESPACE_1, STRING_BAG_ADDR); @@ -115,29 +160,30 @@ private boolean restoreAndTestState(DataInputView in) throws Exception { boolean comparisonRes = true; - for(String key: statePerKey.keySet()) { + for (String key : statePerKey.keySet()) { comparisonRes &= checkStateForKey(key); } // restore the timers Map> restoredTimersPerKey = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder); - if(activeTimers.size() != restoredTimersPerKey.size()) { + if (activeTimers.size() != restoredTimersPerKey.size()) { return false; } - for(String key: statePerKey.keySet()) { + for (String key : statePerKey.keySet()) { Set originalTimers = activeTimers.get(key); Set restoredTimers = restoredTimersPerKey.get(key); comparisonRes &= checkTimersForKey(originalTimers, restoredTimers); } // restore the state - Map> restoredPerKeyState = StateCheckpointUtils.decodeState(reader, combiner.asKeyedFn(), keyCoder, windowCoder, userClassloader); - if(restoredPerKeyState.size() != statePerKey.size()) { + Map> restoredPerKeyState = + StateCheckpointUtils.decodeState(reader, OutputTimeFns.outputAtEarliestInputTimestamp(), keyCoder, windowCoder, userClassloader); + if (restoredPerKeyState.size() != statePerKey.size()) { return false; } - for(String key: statePerKey.keySet()) { + for (String key : statePerKey.keySet()) { FlinkStateInternals originalState = statePerKey.get(key); FlinkStateInternals restoredState = restoredPerKeyState.get(key); comparisonRes &= checkStateForKey(originalState, restoredState); @@ -149,32 +195,32 @@ private boolean checkStateForKey(String key) throws CannotProvideCoderException FlinkStateInternals state = statePerKey.get(key); ValueState value = state.state(NAMESPACE_1, STRING_VALUE_ADDR); - boolean comp = value.get().read().equals("test"); + boolean comp = value.read().equals("test"); ValueState value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR); - comp &= value2.get().read().equals(5); + comp &= value2.read().equals(5); - CombiningValueState combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR); - comp &= combiningValue.get().read().equals(3); + AccumulatorCombiningState combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR); + comp &= combiningValue.read().equals(3); - WatermarkStateInternal watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR); - comp &= watermark.get().read().equals(new Instant(1000)); + WatermarkHoldState watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + comp &= watermark.read().equals(new Instant(1000)); BagState bag = state.state(NAMESPACE_1, STRING_BAG_ADDR); - Iterator it = bag.get().read().iterator(); + Iterator it = bag.read().iterator(); int i = 0; - while(it.hasNext()) { - comp &= it.next().equals("v"+ (++i)); + while (it.hasNext()) { + comp &= it.next().equals("v" + (++i)); } return comp; } - private void storeState(StateBackend.CheckpointStateOutputView out) throws Exception { + private void storeState(AbstractStateBackend.CheckpointStateOutputView out) throws Exception { StateCheckpointWriter checkpointBuilder = StateCheckpointWriter.create(out); Coder keyCoder = StringUtf8Coder.of(); // checkpoint the timers - StateCheckpointUtils.encodeTimers(activeTimers, checkpointBuilder,keyCoder); + StateCheckpointUtils.encodeTimers(activeTimers, checkpointBuilder, keyCoder); // checkpoint the state StateCheckpointUtils.encodeState(statePerKey, checkpointBuilder, keyCoder); @@ -182,46 +228,46 @@ private void storeState(StateBackend.CheckpointStateOutputView out) throws Excep private boolean checkTimersForKey(Set originalTimers, Set restoredTimers) { boolean comp = true; - if(restoredTimers == null) { + if (restoredTimers == null) { return false; } - if(originalTimers.size() != restoredTimers.size()) { + if (originalTimers.size() != restoredTimers.size()) { return false; } - for(TimerInternals.TimerData timer: originalTimers) { + for (TimerInternals.TimerData timer : originalTimers) { comp &= restoredTimers.contains(timer); } return comp; } private boolean checkStateForKey(FlinkStateInternals originalState, FlinkStateInternals restoredState) throws CannotProvideCoderException { - if(restoredState == null) { + if (restoredState == null) { return false; } ValueState orValue = originalState.state(NAMESPACE_1, STRING_VALUE_ADDR); ValueState resValue = restoredState.state(NAMESPACE_1, STRING_VALUE_ADDR); - boolean comp = orValue.get().read().equals(resValue.get().read()); + boolean comp = orValue.read().equals(resValue.read()); ValueState orIntValue = originalState.state(NAMESPACE_1, INT_VALUE_ADDR); ValueState resIntValue = restoredState.state(NAMESPACE_1, INT_VALUE_ADDR); - comp &= orIntValue.get().read().equals(resIntValue.get().read()); + comp &= orIntValue.read().equals(resIntValue.read()); - CombiningValueState combOrValue = originalState.state(NAMESPACE_1, SUM_INTEGER_ADDR); - CombiningValueState combResValue = restoredState.state(NAMESPACE_1, SUM_INTEGER_ADDR); - comp &= combOrValue.get().read().equals(combResValue.get().read()); + AccumulatorCombiningState combOrValue = originalState.state(NAMESPACE_1, SUM_INTEGER_ADDR); + AccumulatorCombiningState combResValue = restoredState.state(NAMESPACE_1, SUM_INTEGER_ADDR); + comp &= combOrValue.read().equals(combResValue.read()); - WatermarkStateInternal orWatermark = originalState.state(NAMESPACE_1, WATERMARK_BAG_ADDR); - WatermarkStateInternal resWatermark = restoredState.state(NAMESPACE_1, WATERMARK_BAG_ADDR); - comp &= orWatermark.get().read().equals(resWatermark.get().read()); + WatermarkHoldState orWatermark = originalState.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + WatermarkHoldState resWatermark = restoredState.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + comp &= orWatermark.read().equals(resWatermark.read()); BagState orBag = originalState.state(NAMESPACE_1, STRING_BAG_ADDR); BagState resBag = restoredState.state(NAMESPACE_1, STRING_BAG_ADDR); - Iterator orIt = orBag.get().read().iterator(); - Iterator resIt = resBag.get().read().iterator(); + Iterator orIt = orBag.read().iterator(); + Iterator resIt = resBag.read().iterator(); while (orIt.hasNext() && resIt.hasNext()) { comp &= orIt.next().equals(resIt.next()); @@ -232,10 +278,10 @@ private boolean checkStateForKey(FlinkStateInternals originalState, Flin private FlinkStateInternals createState(String key) throws CannotProvideCoderException { return new FlinkStateInternals<>( - key, - StringUtf8Coder.of(), - IntervalWindow.getCoder(), - combiner.asKeyedFn()); + key, + StringUtf8Coder.of(), + IntervalWindow.getCoder(), + OutputTimeFns.outputAtEarliestInputTimestamp()); } @Test @@ -243,13 +289,13 @@ public void test() throws Exception { StateSerializationTest test = new StateSerializationTest(); test.initializeStateAndTimers(); - MemoryStateBackend.MemoryCheckpointOutputStream memBackend = new MemoryStateBackend.MemoryCheckpointOutputStream(25728); - StateBackend.CheckpointStateOutputView out = new StateBackend.CheckpointStateOutputView(memBackend); + MemoryStateBackend.MemoryCheckpointOutputStream memBackend = new MemoryStateBackend.MemoryCheckpointOutputStream(32048); + AbstractStateBackend.CheckpointStateOutputView out = new AbstractStateBackend.CheckpointStateOutputView(memBackend); test.storeState(out); byte[] contents = memBackend.closeAndGetBytes(); - ByteArrayInputView in = new ByteArrayInputView(contents); + DataInputView in = new DataInputDeserializer(contents, 0, contents.length); assertEquals(test.restoreAndTestState(in), true); } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java index 1c800fa1ac0c..2990e1f5e263 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java @@ -69,8 +69,8 @@ protected void testProgram() throws Exception { Pipeline p = FlinkTestPipeline.createForStreaming(); - long now = System.currentTimeMillis() + 10000; - System.out.println((now + 5000) / 1000); + Long now = (System.currentTimeMillis() + 10000) / 1000; + System.out.println(now); PCollection> output = p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set @@ -104,7 +104,7 @@ protected void testProgram() throws Exception { @Override public void processElement(ProcessContext c) throws Exception { TableRow row = c.element(); - long timestamp = (Long) row.get("timestamp"); + long timestamp = (Integer) row.get("timestamp"); String userName = (String) row.get("contributor_username"); if (userName != null) { // Sets the timestamp field to be used in windowing. @@ -119,6 +119,7 @@ public void processElement(ProcessContext c) throws Exception { @Override public void processElement(ProcessContext c) { + // TODO: Vacuously true. if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * 1.0) { c.output(c.element()); } From cb3d03092e69cfaa31dfb67e8151dcf7a951b509 Mon Sep 17 00:00:00 2001 From: Max Date: Wed, 2 Mar 2016 12:09:05 +0100 Subject: [PATCH 137/149] [runner] some small fixes for 1.5 - fix AvroITCase - timestamp set correctly - fix advancing of reader in UnboundedSourceWrapper - fix final watermark - fix pom file - cleanup - re-add JavaDoc --- runners/flink/pom.xml | 14 ++- .../FlinkPipelineExecutionEnvironment.java | 14 ++- .../flink/dataflow/FlinkPipelineOptions.java | 3 +- .../FlinkStreamingTransformTranslators.java | 6 +- .../FlinkGroupAlsoByWindowWrapper.java | 7 +- .../streaming/io/UnboundedSourceWrapper.java | 30 ++++-- .../streaming/state/FlinkStateInternals.java | 9 +- .../flink/dataflow/AvroITCase.java | 99 +++++++++++++++++++ .../streaming/TopWikipediaSessionsITCase.java | 18 +--- 9 files changed, 149 insertions(+), 51 deletions(-) create mode 100644 runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index acf79d19ab78..0234ddddc838 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -57,11 +57,9 @@ https://repository.apache.org/content/repositories/snapshots/ false - never - false - never + true @@ -110,6 +108,11 @@ flink-connector-kafka-0.8_2.10 ${flink.version} + + org.apache.flink + flink-avro + ${flink.version} + com.google.cloud.dataflow google-cloud-dataflow-java-sdk-all @@ -127,11 +130,6 @@ 1.9.5 test - - com.google.guava - guava - 19.0 - diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java index 31b3ae562b1b..796849d71666 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java @@ -231,15 +231,19 @@ private void createStreamExecutionEnvironment() { // set parallelism in the options (required by some execution code) options.setParallelism(flinkStreamEnv.getParallelism()); - // although we do not use the generated timestamps, - // enabling timestamps is needed for the watermarks. - // this.flinkStreamEnv.getConfig().enableTimestamps(); + // default to event time this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); // for the following 2 parameters, a value of -1 means that Flink will use // the default values as specified in the configuration. - this.flinkStreamEnv.setNumberOfExecutionRetries(options.getNumberOfExecutionRetries()); - this.flinkStreamEnv.getConfig().setExecutionRetryDelay(options.getExecutionRetryDelay()); + int numRetries = options.getNumberOfExecutionRetries(); + if (numRetries != -1) { + this.flinkStreamEnv.setNumberOfExecutionRetries(numRetries); + } + long retryDelay = options.getExecutionRetryDelay(); + if (retryDelay != -1) { + this.flinkStreamEnv.getConfig().setExecutionRetryDelay(retryDelay); + } // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink). // If the value is not -1, then the validity checks are applied. diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java index 6c5733e86689..2429caca8153 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java @@ -85,8 +85,7 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp void setNumberOfExecutionRetries(Integer retries); @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.") - // TODO: Should really be -1 and the default set correctly. - @Default.Long(10L) + @Default.Long(-1L) Long getExecutionRetryDelay(); void setExecutionRetryDelay(Long delay); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java index 1be51aee4787..99dbedbbeaf3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java @@ -204,7 +204,7 @@ public void translateNode(ParDo.Bound transform, FlinkStreamingTranslat FlinkParDoBoundWrapper doFnWrapper = new FlinkParDoBoundWrapper<>( context.getPipelineOptions(), windowingStrategy, transform.getFn()); DataStream> inputDataStream = context.getInputDataStream(context.getInput(transform)); - SingleOutputStreamOperator, ?> outDataStream = inputDataStream.flatMap(doFnWrapper) + SingleOutputStreamOperator> outDataStream = inputDataStream.flatMap(doFnWrapper) .returns(outputWindowedValueCoder); context.setOutputDataStream(context.getOutput(transform), outDataStream); @@ -232,7 +232,7 @@ public void translateNode(Window.Bound transform, FlinkStreamingTranslationCo final FlinkParDoBoundWrapper windowDoFnAssigner = new FlinkParDoBoundWrapper<>( context.getPipelineOptions(), windowingStrategy, createWindowAssigner(windowFn)); - SingleOutputStreamOperator, ?> windowedStream = + SingleOutputStreamOperator> windowedStream = inputDataStream.flatMap(windowDoFnAssigner).returns(outputWindowedValueCoder); context.setOutputDataStream(context.getOutput(transform), windowedStream); } @@ -349,7 +349,7 @@ public void translateNode(ParDo.BoundMulti transform, FlinkStreamingTra transform.getMainOutputTag(), tagsToLabels); DataStream> inputDataStream = context.getInputDataStream(context.getInput(transform)); - SingleOutputStreamOperator, ?> intermDataStream = + SingleOutputStreamOperator> intermDataStream = inputDataStream.flatMap(doFnWrapper).returns(intermWindowedValueCoder); for (Map.Entry, PCollection> output : outputs.entrySet()) { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java index a003868887ad..fb3d329990dd 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -43,10 +43,8 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTaskState; -import org.apache.flink.util.Collector; import org.joda.time.Instant; -import javax.annotation.Nullable; import java.io.IOException; import java.util.*; @@ -309,7 +307,6 @@ public void processWatermark(Watermark mark) throws Exception { @Override public void close() throws Exception { - processWatermark(new Watermark(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis())); super.close(); } @@ -517,9 +514,9 @@ public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() @Override public void outputWindowedValue(KV output, Instant timestamp, Collection windows, PaneInfo pane) { // TODO: No need to represent timestamp twice. - // collector.setAbsoluteTimestamp(timestamp.getMillis()); - collector.setAbsoluteTimestamp(0); + collector.setAbsoluteTimestamp(timestamp.getMillis()); collector.collect(WindowedValue.of(output, timestamp, windows, pane)); + } @Override diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index dae2dc21f7eb..e065f873673f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -72,23 +72,33 @@ public void run(SourceContext> ctx) throws Exception { context = (StreamSource.ManualWatermarkContext>) ctx; runtime = (StreamingRuntimeContext) getRuntimeContext(); - this.isRunning = reader.start(); + this.isRunning = true; + boolean inputAvailable = reader.start(); + setNextWatermarkTimer(this.runtime); while (isRunning) { - // get it and its timestamp from the source - T item = reader.getCurrent(); - Instant timestamp = reader.getCurrentTimestamp(); + while (!inputAvailable && isRunning) { + // wait a bit until we retry to pull more records + Thread.sleep(50); + inputAvailable = reader.advance(); + } + + if (inputAvailable) { + + // get it and its timestamp from the source + T item = reader.getCurrent(); + Instant timestamp = reader.getCurrentTimestamp(); + + // write it to the output collector + synchronized (ctx.getCheckpointLock()) { + context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis()); + } - // write it to the output collector - synchronized (ctx.getCheckpointLock()) { - context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis()); + inputAvailable = reader.advance(); } - // TODO: This will run the loop only until the underlying reader first indicates input has stalled. - // try to go to the next record - this.isRunning = reader.advance(); } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java index 5b7537ef34c7..41ab5f0bc07e 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java @@ -32,8 +32,13 @@ import java.io.IOException; import java.util.*; -// TODO: State should be rewritten to redirect to Flink per-key state so that coders and combiners don't need -// to be serialized along with encoded values when snapshotting. +/** + * An implementation of the Beam {@link StateInternals}. This implementation simply keeps elements in memory. + * This state is periodically checkpointed by Flink, for fault-tolerance. + * + * TODO: State should be rewritten to redirect to Flink per-key state so that coders and combiners don't need + * to be serialized along with encoded values when snapshotting. + */ public class FlinkStateInternals implements StateInternals { private final K key; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java new file mode 100644 index 000000000000..ce53d44dc773 --- /dev/null +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java @@ -0,0 +1,99 @@ +/* + * Copyright 2015 Data Artisans GmbH + * + * Licensed 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 com.dataartisans.flink.dataflow; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.io.AvroIO; +import com.google.cloud.dataflow.sdk.io.TextIO; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.common.base.Joiner; +import org.apache.flink.api.io.avro.example.User; +import org.apache.flink.test.util.JavaProgramTestBase; + + +public class AvroITCase extends JavaProgramTestBase { + + protected String resultPath; + protected String tmpPath; + + public AvroITCase(){ + } + + static final String[] EXPECTED_RESULT = new String[] { + "Joe red 3", + "Mary blue 4", + "Mark green 1", + "Julia purple 5" + }; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + tmpPath = getTempDirPath("tmp"); + + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } + + @Override + protected void testProgram() throws Exception { + runProgram(tmpPath, resultPath); + } + + private static void runProgram(String tmpPath, String resultPath) { + Pipeline p = FlinkTestPipeline.createForBatch(); + + p + .apply(Create.of( + new User("Joe", 3, "red"), + new User("Mary", 4, "blue"), + new User("Mark", 1, "green"), + new User("Julia", 5, "purple")) + .withCoder(AvroCoder.of(User.class))) + + .apply(AvroIO.Write.to(tmpPath) + .withSchema(User.class)); + + p.run(); + + p = FlinkTestPipeline.createForBatch(); + + p + .apply(AvroIO.Read.from(tmpPath).withSchema(User.class).withoutValidation()) + + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + User u = c.element(); + String result = u.getName() + " " + u.getFavoriteColor() + " " + u.getFavoriteNumber(); + c.output(result); + } + })) + + .apply(TextIO.Write.to(resultPath)); + + p.run(); + } + +} + diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java index 2990e1f5e263..ad5b53a58ebe 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java @@ -70,7 +70,6 @@ protected void testProgram() throws Exception { Pipeline p = FlinkTestPipeline.createForStreaming(); Long now = (System.currentTimeMillis() + 10000) / 1000; - System.out.println(now); PCollection> output = p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set @@ -113,28 +112,15 @@ public void processElement(ProcessContext c) throws Exception { } })) - .apply(ParDo.named("SampleUsers").of( - new DoFn() { - private static final long serialVersionUID = 0; + .apply(Window.into(Sessions.withGapDuration(Duration.standardMinutes(1)))) - @Override - public void processElement(ProcessContext c) { - // TODO: Vacuously true. - if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * 1.0) { - c.output(c.element()); - } - } - })) - - .apply(Window.into(Sessions.withGapDuration(Duration.standardMinutes(1)))) - .apply(Count.perElement()); + .apply(Count.perElement()); PCollection format = output.apply(ParDo.of(new DoFn, String>() { @Override public void processElement(ProcessContext c) throws Exception { KV el = c.element(); String out = "user: " + el.getKey() + " value:" + el.getValue(); - System.out.println(out); c.output(out); } })); From 699562d84d9d1b0aecd64fbbeca34724f37522ea Mon Sep 17 00:00:00 2001 From: kl0u Date: Wed, 2 Mar 2016 16:09:53 +0100 Subject: [PATCH 138/149] Fixes the GroupAlsoByWindowTest. --- .../streaming/GroupAlsoByWindowTest.java | 70 ++++++++++--------- 1 file changed, 36 insertions(+), 34 deletions(-) diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java index 434f82758a0a..01f9c323d52e 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java @@ -116,7 +116,7 @@ public void testWithLateness() throws Exception { new Instant(initialTime + 1), new IntervalWindow(new Instant(0), new Instant(2000)), PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) - , initialTime)); + , initialTime + 1)); expectedOutput.add(new Watermark(initialTime + 2000)); expectedOutput.add(new StreamRecord<>( @@ -124,14 +124,15 @@ public void testWithLateness() throws Exception { new Instant(initialTime + 1999), new IntervalWindow(new Instant(0), new Instant(2000)), PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)) - , initialTime)); + , initialTime + 1999)); + expectedOutput.add(new StreamRecord<>( WindowedValue.of(KV.of("key1", 6), new Instant(initialTime + 1999), new IntervalWindow(new Instant(0), new Instant(2000)), PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 2, 2)) - , initialTime)); + , initialTime + 1999)); expectedOutput.add(new Watermark(initialTime + 4000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); @@ -181,7 +182,7 @@ public void testSessionWindows() throws Exception { new Instant(initialTime + 1), new IntervalWindow(new Instant(1), new Instant(5700)), PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) - , initialTime)); + , initialTime + 1)); expectedOutput.add(new Watermark(initialTime + 6000)); expectedOutput.add(new StreamRecord<>( @@ -189,7 +190,7 @@ public void testSessionWindows() throws Exception { new Instant(initialTime + 6700), new IntervalWindow(new Instant(1), new Instant(10900)), PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) - , initialTime)); + , initialTime + 6700)); expectedOutput.add(new Watermark(initialTime + 12000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); @@ -210,13 +211,13 @@ public void testSlidingWindows() throws Exception { new Instant(initialTime + 5000), new IntervalWindow(new Instant(0), new Instant(10000)), PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime)); + , initialTime + 5000)); expectedOutput.add(new StreamRecord<>( WindowedValue.of(KV.of("key1", 6), new Instant(initialTime + 1), new IntervalWindow(new Instant(-5000), new Instant(5000)), PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime)); + , initialTime + 1)); expectedOutput.add(new Watermark(initialTime + 10000)); expectedOutput.add(new StreamRecord<>( @@ -224,19 +225,19 @@ public void testSlidingWindows() throws Exception { new Instant(initialTime + 15000), new IntervalWindow(new Instant(10000), new Instant(20000)), PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime)); + , initialTime + 15000)); expectedOutput.add(new StreamRecord<>( WindowedValue.of(KV.of("key1", 3), new Instant(initialTime + 10000), new IntervalWindow(new Instant(5000), new Instant(15000)), PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime)); + , initialTime + 10000)); expectedOutput.add(new StreamRecord<>( WindowedValue.of(KV.of("key2", 1), new Instant(initialTime + 19500), new IntervalWindow(new Instant(10000), new Instant(20000)), PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime)); + , initialTime + 19500)); expectedOutput.add(new Watermark(initialTime + 20000)); expectedOutput.add(new StreamRecord<>( @@ -250,13 +251,13 @@ public void testSlidingWindows() throws Exception { */ new IntervalWindow(new Instant(15000), new Instant(25000)), PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime)); + , initialTime + 20000)); expectedOutput.add(new StreamRecord<>( WindowedValue.of(KV.of("key1", 8), new Instant(initialTime + 20000), new IntervalWindow(new Instant(15000), new Instant(25000)), PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime)); + , initialTime + 20000)); expectedOutput.add(new Watermark(initialTime + 25000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); @@ -272,13 +273,13 @@ public void testAfterWatermarkProgram() throws Exception { ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6), - new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime)); + new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 1)); expectedOutput.add(new Watermark(initialTime + 10000)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11), - new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime)); + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 10000)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime)); + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500)); expectedOutput.add(new Watermark(initialTime + 20000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); @@ -295,13 +296,13 @@ public void testAfterCountProgram() throws Exception { ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime)); + new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 1)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime)); + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 10000)); expectedOutput.add(new Watermark(initialTime + 10000)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime)); + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime + 19500)); expectedOutput.add(new Watermark(initialTime + 20000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); @@ -326,21 +327,21 @@ public void testCompoundProgram() throws Exception { * */ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime)); + new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime)); + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime)); + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), - new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime)); + new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200)); expectedOutput.add(new Watermark(initialTime + 10000)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime)); + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime)); + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500)); expectedOutput.add(new Watermark(initialTime + 20000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); @@ -348,7 +349,7 @@ public void testCompoundProgram() throws Exception { testHarness.close(); } - // Disabled + @Test public void testCompoundAccumulatingPanesProgram() throws Exception { WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc; long initialTime = 0L; @@ -357,21 +358,21 @@ public void testCompoundAccumulatingPanesProgram() throws Exception { ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime)); + new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime)); + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 10), - new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime)); + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6), - new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime)); + new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200)); expectedOutput.add(new Watermark(initialTime + 10000)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11), - new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime)); + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500)); expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime)); + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500)); expectedOutput.add(new Watermark(initialTime + 20000)); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); @@ -434,11 +435,12 @@ public int compare(Object o1, Object o2) { StreamRecord>> sr0 = (StreamRecord>>) o1; StreamRecord>> sr1 = (StreamRecord>>) o2; - if (sr0.getTimestamp() != sr1.getTimestamp()) { - return (int) (sr0.getTimestamp() - sr1.getTimestamp()); + int comparison = (int) (sr0.getValue().getTimestamp().getMillis() - sr1.getValue().getTimestamp().getMillis()); + if (comparison != 0) { + return comparison; } - int comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey()); + comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey()); if(comparison == 0) { comparison = Integer.compare( sr0.getValue().getValue().getValue(), From e0b6782c80724b9b7fe391eceae7e85058a58274 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 3 Mar 2016 11:30:53 +0100 Subject: [PATCH 139/149] [travis] install snapshot version of SDK before running CI --- runners/flink/.travis.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/runners/flink/.travis.yml b/runners/flink/.travis.yml index d7fbf5be9438..4780b56c0bd3 100644 --- a/runners/flink/.travis.yml +++ b/runners/flink/.travis.yml @@ -9,4 +9,6 @@ jdk: # not supported yet :( #- openjdk8 -script: mvn verify +install: (git clone --depth 1 https://github.com/GoogleCloudPlatform/DataflowJavaSDK.git sdk && cd sdk && mvn clean install -DskipTests) + +script: mvn verify \ No newline at end of file From a5876172e753b55ce562d49effd3d52f65dc7eb7 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 3 Mar 2016 14:57:06 +0100 Subject: [PATCH 140/149] [tests] suppress unnecessary log output --- runners/flink/pom.xml | 6 +++++ .../src/test/resources/log4j-test.properties | 27 +++++++++++++++++++ 2 files changed, 33 insertions(+) create mode 100644 runners/flink/src/test/resources/log4j-test.properties diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 0234ddddc838..6b12f2dd7f16 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -174,6 +174,9 @@ + + -Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit + @@ -181,6 +184,9 @@ org.apache.maven.plugins maven-surefire-plugin 2.17 + + -Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit + diff --git a/runners/flink/src/test/resources/log4j-test.properties b/runners/flink/src/test/resources/log4j-test.properties new file mode 100644 index 000000000000..4c74d85d7c62 --- /dev/null +++ b/runners/flink/src/test/resources/log4j-test.properties @@ -0,0 +1,27 @@ +################################################################################ +# 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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +log4j.rootLogger=OFF, testlogger + +# A1 is set to be a ConsoleAppender. +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target = System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n From 07ab1117d2046ddbfcfec8f1520c9ee188fa253a Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 1 Mar 2016 17:57:02 +0100 Subject: [PATCH 141/149] [maven] add project for Runners --- pom.xml | 1 + runners/pom.xml | 43 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 44 insertions(+) create mode 100644 runners/pom.xml diff --git a/pom.xml b/pom.xml index ba130d25a3d2..de47ff5c4fa5 100644 --- a/pom.xml +++ b/pom.xml @@ -91,6 +91,7 @@ pom sdk + runners examples maven-archetypes/starter maven-archetypes/examples diff --git a/runners/pom.xml b/runners/pom.xml new file mode 100644 index 000000000000..757e2081d1cc --- /dev/null +++ b/runners/pom.xml @@ -0,0 +1,43 @@ + + + + + 4.0.0 + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-parent + 1.5.0-SNAPSHOT + + + org.apache.beam + runners + 1.5.0-SNAPSHOT + + pom + + Beam Runners + + + flink + + + From a1bd52b68b477c95411ec938be8e36fca0b24d59 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 4 Mar 2016 15:08:26 +0100 Subject: [PATCH 142/149] [flink] adjust root pom.xml to Beam --- runners/flink/pom.xml | 475 +++++++++++++++++++++--------------------- 1 file changed, 243 insertions(+), 232 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 6b12f2dd7f16..2110c2c8023f 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -1,253 +1,264 @@ + - http://www.apache.org/licenses/LICENSE-2.0 + 4.0.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. + + org.apache.beam + runners + 1.5.0-SNAPSHOT + ---> - - - 4.0.0 - - com.dataartisans - flink-dataflow - 0.3-SNAPSHOT - - Flink Beam Runner - jar - - 2015 - - - - The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt - repo - - - - - UTF-8 - UTF-8 - 1.0-SNAPSHOT - 1.5.0-SNAPSHOT - - com.dataartisans.flink.dataflow.examples.WordCount - kinglear.txt - wordcounts.txt - 1 - - - - - apache.snapshots - Apache Development Snapshot Repository - https://repository.apache.org/content/repositories/snapshots/ - - false - - - true - - - - - - - org.apache.flink - flink-core - ${flink.version} - - - org.apache.flink - flink-streaming-java_2.10 - ${flink.version} - - - org.apache.flink - flink-streaming-java_2.10 - ${flink.version} - test - test-jar - - - org.apache.flink - flink-java - ${flink.version} - - - org.apache.flink - flink-avro_2.10 - ${flink.version} - - - org.apache.flink - flink-clients_2.10 - ${flink.version} - - - org.apache.flink - flink-test-utils_2.10 - ${flink.version} - test - - - org.apache.flink - flink-connector-kafka-0.8_2.10 - ${flink.version} - - - org.apache.flink - flink-avro - ${flink.version} - - - com.google.cloud.dataflow - google-cloud-dataflow-java-sdk-all - ${beam.version} - - - org.slf4j - slf4j-jdk14 - - - - - org.mockito - mockito-all - 1.9.5 - test - - - - - - - - - org.apache.maven.plugins - maven-jar-plugin - 2.6 - - - - true - true - - - - + flink-runner + 0.3-SNAPSHOT - - + Flink Beam Runner + jar - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.7 - 1.7 - - - - - - maven-failsafe-plugin - 2.17 - - - - integration-test - verify - - - + 2015 + + + + The Apache Software License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + UTF-8 + UTF-8 + 1.0-SNAPSHOT + 1.5.0-SNAPSHOT + + org.apache.beam.runners.flink.examples.WordCount + kinglear.txt + wordcounts.txt + 1 + + + + + apache.snapshots + Apache Development Snapshot Repository + https://repository.apache.org/content/repositories/snapshots/ + + false + + + true + + + + + + + org.apache.flink + flink-core + ${flink.version} + + + org.apache.flink + flink-streaming-java_2.10 + ${flink.version} + + + org.apache.flink + flink-streaming-java_2.10 + ${flink.version} + test + test-jar + + + org.apache.flink + flink-java + ${flink.version} + + + org.apache.flink + flink-avro_2.10 + ${flink.version} + + + org.apache.flink + flink-clients_2.10 + ${flink.version} + + + org.apache.flink + flink-test-utils_2.10 + ${flink.version} + test + + + org.apache.flink + flink-connector-kafka-0.8_2.10 + ${flink.version} + + + org.apache.flink + flink-avro + ${flink.version} + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + ${beam.version} + + + org.slf4j + slf4j-jdk14 + + + + + org.mockito + mockito-all + 1.9.5 + test + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.6 + + + + true + true + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.7 + 1.7 + + + + + + maven-failsafe-plugin + 2.17 + + + + integration-test + verify + + + -Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit - + - - - org.apache.maven.plugins - maven-surefire-plugin - 2.17 + + + org.apache.maven.plugins + maven-surefire-plugin + 2.17 -Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit - + - - - org.apache.maven.plugins - maven-eclipse-plugin - 2.8 - - - org.eclipse.jdt.launching.JRE_CONTAINER - - true - true - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - 1.3.1 - - - enforce-maven - - enforce - - - - - [1.7,) - - - - [3.0.3,) - - - - - - - - - org.codehaus.mojo - exec-maven-plugin - 1.2.1 - - java - - -classpath - - ${clazz} - --input=${input} - --output=${output} - --parallelism=${parallelism} - - - + + + org.apache.maven.plugins + maven-eclipse-plugin + 2.8 + + + org.eclipse.jdt.launching.JRE_CONTAINER + + true + true + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.3.1 + + + enforce-maven + + enforce + + + + + [1.7,) + + + + [3.0.3,) + + + + + + + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + + none + + + + java + + -classpath + + ${clazz} + --input=${input} + --output=${output} + --parallelism=${parallelism} + + + - + - + From 0bb73db2770b59fcfc2e0779f7ec8450f37e58b5 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 2 Mar 2016 23:03:37 +0100 Subject: [PATCH 143/149] [flink] convert tabs to 2 spaces --- .../FlinkPipelineExecutionEnvironment.java | 394 ++--- .../flink/dataflow/FlinkPipelineOptions.java | 102 +- .../flink/dataflow/FlinkPipelineRunner.java | 308 ++-- .../flink/dataflow/FlinkRunnerResult.java | 64 +- .../flink/dataflow/examples/TFIDF.java | 720 ++++----- .../flink/dataflow/examples/WordCount.java | 154 +- .../examples/streaming/JoinExamples.java | 208 +-- .../KafkaWindowedWordCountExample.java | 204 +-- .../examples/streaming/WindowedWordCount.java | 160 +- .../flink/dataflow/io/ConsoleIO.java | 82 +- .../FlinkBatchPipelineTranslator.java | 230 +-- .../FlinkBatchTransformTranslators.java | 892 +++++------ .../FlinkBatchTranslationContext.java | 154 +- .../translation/FlinkPipelineTranslator.java | 6 +- .../FlinkStreamingPipelineTranslator.java | 218 +-- .../FlinkStreamingTransformTranslators.java | 684 ++++----- .../FlinkStreamingTranslationContext.java | 88 +- .../FlinkCoGroupKeyedListAggregator.java | 48 +- .../functions/FlinkCreateFunction.java | 50 +- .../functions/FlinkDoFnFunction.java | 300 ++-- .../FlinkKeyedListAggregationFunction.java | 74 +- .../FlinkMultiOutputDoFnFunction.java | 242 +-- .../FlinkMultiOutputPruningFunction.java | 22 +- .../functions/FlinkPartialReduceFunction.java | 50 +- .../functions/FlinkReduceFunction.java | 32 +- .../translation/functions/UnionCoder.java | 218 +-- .../translation/types/CoderComparator.java | 362 ++--- .../types/CoderTypeInformation.java | 162 +- .../types/CoderTypeSerializer.java | 228 +-- .../InspectableByteArrayOutputStream.java | 12 +- .../translation/types/KvCoderComperator.java | 452 +++--- .../types/KvCoderTypeInformation.java | 298 ++-- .../types/VoidCoderTypeSerializer.java | 154 +- .../wrappers/CombineFnAggregatorWrapper.java | 94 +- .../wrappers/DataInputViewWrapper.java | 50 +- .../wrappers/DataOutputViewWrapper.java | 32 +- .../SerializableFnAggregatorWrapper.java | 92 +- .../wrappers/SinkOutputFormat.java | 160 +- .../wrappers/SourceInputFormat.java | 240 +-- .../wrappers/SourceInputSplit.java | 30 +- .../streaming/FlinkAbstractParDoWrapper.java | 432 +++--- .../FlinkGroupAlsoByWindowWrapper.java | 1116 +++++++------- .../streaming/FlinkGroupByKeyWrapper.java | 46 +- .../FlinkParDoBoundMultiWrapper.java | 70 +- .../streaming/FlinkParDoBoundWrapper.java | 102 +- .../io/FlinkStreamingCreateFunction.java | 52 +- .../streaming/io/UnboundedFlinkSource.java | 76 +- .../streaming/io/UnboundedSocketSource.java | 372 ++--- .../streaming/io/UnboundedSourceWrapper.java | 182 +-- .../state/AbstractFlinkTimerInternals.java | 178 +-- .../streaming/state/FlinkStateInternals.java | 1338 ++++++++--------- .../state/StateCheckpointReader.java | 122 +- .../streaming/state/StateCheckpointUtils.java | 232 +-- .../state/StateCheckpointWriter.java | 198 +-- .../wrappers/streaming/state/StateType.java | 68 +- .../flink/dataflow/AvroITCase.java | 128 +- .../flink/dataflow/FlattenizeITCase.java | 60 +- .../flink/dataflow/FlinkTestPipeline.java | 74 +- .../flink/dataflow/JoinExamplesITCase.java | 122 +- .../flink/dataflow/MaybeEmptyTestITCase.java | 64 +- .../dataflow/ParDoMultiOutputITCase.java | 128 +- .../flink/dataflow/ReadSourceITCase.java | 244 +-- .../dataflow/RemoveDuplicatesEmptyITCase.java | 48 +- .../dataflow/RemoveDuplicatesITCase.java | 50 +- .../flink/dataflow/SideInputITCase.java | 52 +- .../flink/dataflow/TfIdfITCase.java | 58 +- .../flink/dataflow/WordCountITCase.java | 54 +- .../flink/dataflow/WordCountJoin2ITCase.java | 196 +-- .../flink/dataflow/WordCountJoin3ITCase.java | 234 +-- .../flink/dataflow/WriteSinkITCase.java | 218 +-- .../streaming/GroupAlsoByWindowTest.java | 920 ++++++------ .../streaming/GroupByNullKeyTest.java | 138 +- .../streaming/StateSerializationTest.java | 506 +++---- .../streaming/TopWikipediaSessionsITCase.java | 178 +-- .../flink/dataflow/util/JoinExamples.java | 232 +-- 75 files changed, 8179 insertions(+), 8179 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java index 796849d71666..c2139c68b47a 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java @@ -40,228 +40,228 @@ */ public class FlinkPipelineExecutionEnvironment { - private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class); + private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class); - private final FlinkPipelineOptions options; + private final FlinkPipelineOptions options; - /** - * The Flink Batch execution environment. This is instantiated to either a - * {@link org.apache.flink.api.java.CollectionEnvironment}, - * a {@link org.apache.flink.api.java.LocalEnvironment} or - * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration - * options. - */ - private ExecutionEnvironment flinkBatchEnv; + /** + * The Flink Batch execution environment. This is instantiated to either a + * {@link org.apache.flink.api.java.CollectionEnvironment}, + * a {@link org.apache.flink.api.java.LocalEnvironment} or + * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration + * options. + */ + private ExecutionEnvironment flinkBatchEnv; - /** - * The Flink Streaming execution environment. This is instantiated to either a - * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or - * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending - * on the configuration options, and more specifically, the url of the master. - */ - private StreamExecutionEnvironment flinkStreamEnv; + /** + * The Flink Streaming execution environment. This is instantiated to either a + * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or + * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending + * on the configuration options, and more specifically, the url of the master. + */ + private StreamExecutionEnvironment flinkStreamEnv; - /** - * Translator for this FlinkPipelineRunner. Its role is to translate the Beam operators to - * their Flink counterparts. Based on the options provided by the user, if we have a streaming job, - * this is instantiated as a {@link FlinkStreamingPipelineTranslator}. In other case, i.e. a batch job, - * a {@link FlinkBatchPipelineTranslator} is created. - */ - private FlinkPipelineTranslator flinkPipelineTranslator; + /** + * Translator for this FlinkPipelineRunner. Its role is to translate the Beam operators to + * their Flink counterparts. Based on the options provided by the user, if we have a streaming job, + * this is instantiated as a {@link FlinkStreamingPipelineTranslator}. In other case, i.e. a batch job, + * a {@link FlinkBatchPipelineTranslator} is created. + */ + private FlinkPipelineTranslator flinkPipelineTranslator; - /** - * Creates a {@link FlinkPipelineExecutionEnvironment} with the user-specified parameters in the - * provided {@link FlinkPipelineOptions}. - * - * @param options the user-defined pipeline options. - * */ - public FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) { - this.options = Preconditions.checkNotNull(options); - this.createPipelineExecutionEnvironment(); - this.createPipelineTranslator(); - } + /** + * Creates a {@link FlinkPipelineExecutionEnvironment} with the user-specified parameters in the + * provided {@link FlinkPipelineOptions}. + * + * @param options the user-defined pipeline options. + * */ + public FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) { + this.options = Preconditions.checkNotNull(options); + this.createPipelineExecutionEnvironment(); + this.createPipelineTranslator(); + } - /** - * Depending on the type of job (Streaming or Batch) and the user-specified options, - * this method creates the adequate ExecutionEnvironment. - */ - private void createPipelineExecutionEnvironment() { - if (options.isStreaming()) { - createStreamExecutionEnvironment(); - } else { - createBatchExecutionEnvironment(); - } - } + /** + * Depending on the type of job (Streaming or Batch) and the user-specified options, + * this method creates the adequate ExecutionEnvironment. + */ + private void createPipelineExecutionEnvironment() { + if (options.isStreaming()) { + createStreamExecutionEnvironment(); + } else { + createBatchExecutionEnvironment(); + } + } - /** - * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph - * translator. In the case of batch, it will work with {@link org.apache.flink.api.java.DataSet}, - * while for streaming, it will work with {@link org.apache.flink.streaming.api.datastream.DataStream}. - */ - private void createPipelineTranslator() { - checkInitializationState(); - if (this.flinkPipelineTranslator != null) { - throw new IllegalStateException("FlinkPipelineTranslator already initialized."); - } + /** + * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph + * translator. In the case of batch, it will work with {@link org.apache.flink.api.java.DataSet}, + * while for streaming, it will work with {@link org.apache.flink.streaming.api.datastream.DataStream}. + */ + private void createPipelineTranslator() { + checkInitializationState(); + if (this.flinkPipelineTranslator != null) { + throw new IllegalStateException("FlinkPipelineTranslator already initialized."); + } - this.flinkPipelineTranslator = options.isStreaming() ? - new FlinkStreamingPipelineTranslator(flinkStreamEnv, options) : - new FlinkBatchPipelineTranslator(flinkBatchEnv, options); - } + this.flinkPipelineTranslator = options.isStreaming() ? + new FlinkStreamingPipelineTranslator(flinkStreamEnv, options) : + new FlinkBatchPipelineTranslator(flinkBatchEnv, options); + } - /** - * Depending on if the job is a Streaming or a Batch one, this method creates - * the necessary execution environment and pipeline translator, and translates - * the {@link com.google.cloud.dataflow.sdk.values.PCollection} program into - * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream} - * one. - * */ - public void translate(Pipeline pipeline) { - checkInitializationState(); - if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) { - createPipelineExecutionEnvironment(); - } - if (this.flinkPipelineTranslator == null) { - createPipelineTranslator(); - } - this.flinkPipelineTranslator.translate(pipeline); - } + /** + * Depending on if the job is a Streaming or a Batch one, this method creates + * the necessary execution environment and pipeline translator, and translates + * the {@link com.google.cloud.dataflow.sdk.values.PCollection} program into + * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream} + * one. + * */ + public void translate(Pipeline pipeline) { + checkInitializationState(); + if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) { + createPipelineExecutionEnvironment(); + } + if (this.flinkPipelineTranslator == null) { + createPipelineTranslator(); + } + this.flinkPipelineTranslator.translate(pipeline); + } - /** - * Launches the program execution. - * */ - public JobExecutionResult executePipeline() throws Exception { - if (options.isStreaming()) { - if (this.flinkStreamEnv == null) { - throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized."); - } - if (this.flinkPipelineTranslator == null) { - throw new RuntimeException("FlinkPipelineTranslator not initialized."); - } - return this.flinkStreamEnv.execute(); - } else { - if (this.flinkBatchEnv == null) { - throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized."); - } - if (this.flinkPipelineTranslator == null) { - throw new RuntimeException("FlinkPipelineTranslator not initialized."); - } - return this.flinkBatchEnv.execute(); - } - } + /** + * Launches the program execution. + * */ + public JobExecutionResult executePipeline() throws Exception { + if (options.isStreaming()) { + if (this.flinkStreamEnv == null) { + throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized."); + } + if (this.flinkPipelineTranslator == null) { + throw new RuntimeException("FlinkPipelineTranslator not initialized."); + } + return this.flinkStreamEnv.execute(); + } else { + if (this.flinkBatchEnv == null) { + throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized."); + } + if (this.flinkPipelineTranslator == null) { + throw new RuntimeException("FlinkPipelineTranslator not initialized."); + } + return this.flinkBatchEnv.execute(); + } + } - /** - * If the submitted job is a batch processing job, this method creates the adequate - * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending - * on the user-specified options. - */ - private void createBatchExecutionEnvironment() { - if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) { - throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized."); - } + /** + * If the submitted job is a batch processing job, this method creates the adequate + * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending + * on the user-specified options. + */ + private void createBatchExecutionEnvironment() { + if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) { + throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized."); + } - LOG.info("Creating the required Batch Execution Environment."); + LOG.info("Creating the required Batch Execution Environment."); - String masterUrl = options.getFlinkMaster(); - this.flinkStreamEnv = null; + String masterUrl = options.getFlinkMaster(); + this.flinkStreamEnv = null; - // depending on the master, create the right environment. - if (masterUrl.equals("[local]")) { - this.flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment(); - } else if (masterUrl.equals("[collection]")) { - this.flinkBatchEnv = new CollectionEnvironment(); - } else if (masterUrl.equals("[auto]")) { - this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment(); - } else if (masterUrl.matches(".*:\\d*")) { - String[] parts = masterUrl.split(":"); - List stagingFiles = options.getFilesToStage(); - this.flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0], - Integer.parseInt(parts[1]), - stagingFiles.toArray(new String[stagingFiles.size()])); - } else { - LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl); - this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment(); - } + // depending on the master, create the right environment. + if (masterUrl.equals("[local]")) { + this.flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment(); + } else if (masterUrl.equals("[collection]")) { + this.flinkBatchEnv = new CollectionEnvironment(); + } else if (masterUrl.equals("[auto]")) { + this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment(); + } else if (masterUrl.matches(".*:\\d*")) { + String[] parts = masterUrl.split(":"); + List stagingFiles = options.getFilesToStage(); + this.flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0], + Integer.parseInt(parts[1]), + stagingFiles.toArray(new String[stagingFiles.size()])); + } else { + LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl); + this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment(); + } - // set the correct parallelism. - if (options.getParallelism() != -1 && !(this.flinkBatchEnv instanceof CollectionEnvironment)) { - this.flinkBatchEnv.setParallelism(options.getParallelism()); - } + // set the correct parallelism. + if (options.getParallelism() != -1 && !(this.flinkBatchEnv instanceof CollectionEnvironment)) { + this.flinkBatchEnv.setParallelism(options.getParallelism()); + } - // set parallelism in the options (required by some execution code) - options.setParallelism(flinkBatchEnv.getParallelism()); - } + // set parallelism in the options (required by some execution code) + options.setParallelism(flinkBatchEnv.getParallelism()); + } - /** - * If the submitted job is a stream processing job, this method creates the adequate - * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending - * on the user-specified options. - */ - private void createStreamExecutionEnvironment() { - if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) { - throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized."); - } + /** + * If the submitted job is a stream processing job, this method creates the adequate + * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending + * on the user-specified options. + */ + private void createStreamExecutionEnvironment() { + if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) { + throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized."); + } - LOG.info("Creating the required Streaming Environment."); + LOG.info("Creating the required Streaming Environment."); - String masterUrl = options.getFlinkMaster(); - this.flinkBatchEnv = null; + String masterUrl = options.getFlinkMaster(); + this.flinkBatchEnv = null; - // depending on the master, create the right environment. - if (masterUrl.equals("[local]")) { - this.flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment(); - } else if (masterUrl.equals("[auto]")) { - this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - } else if (masterUrl.matches(".*:\\d*")) { - String[] parts = masterUrl.split(":"); - List stagingFiles = options.getFilesToStage(); - this.flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0], - Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()])); - } else { - LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl); - this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - } + // depending on the master, create the right environment. + if (masterUrl.equals("[local]")) { + this.flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment(); + } else if (masterUrl.equals("[auto]")) { + this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + } else if (masterUrl.matches(".*:\\d*")) { + String[] parts = masterUrl.split(":"); + List stagingFiles = options.getFilesToStage(); + this.flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0], + Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()])); + } else { + LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl); + this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + } - // set the correct parallelism. - if (options.getParallelism() != -1) { - this.flinkStreamEnv.setParallelism(options.getParallelism()); - } + // set the correct parallelism. + if (options.getParallelism() != -1) { + this.flinkStreamEnv.setParallelism(options.getParallelism()); + } - // set parallelism in the options (required by some execution code) - options.setParallelism(flinkStreamEnv.getParallelism()); + // set parallelism in the options (required by some execution code) + options.setParallelism(flinkStreamEnv.getParallelism()); - // default to event time - this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); + // default to event time + this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); - // for the following 2 parameters, a value of -1 means that Flink will use - // the default values as specified in the configuration. - int numRetries = options.getNumberOfExecutionRetries(); - if (numRetries != -1) { - this.flinkStreamEnv.setNumberOfExecutionRetries(numRetries); - } - long retryDelay = options.getExecutionRetryDelay(); - if (retryDelay != -1) { - this.flinkStreamEnv.getConfig().setExecutionRetryDelay(retryDelay); - } + // for the following 2 parameters, a value of -1 means that Flink will use + // the default values as specified in the configuration. + int numRetries = options.getNumberOfExecutionRetries(); + if (numRetries != -1) { + this.flinkStreamEnv.setNumberOfExecutionRetries(numRetries); + } + long retryDelay = options.getExecutionRetryDelay(); + if (retryDelay != -1) { + this.flinkStreamEnv.getConfig().setExecutionRetryDelay(retryDelay); + } - // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink). - // If the value is not -1, then the validity checks are applied. - // By default, checkpointing is disabled. - long checkpointInterval = options.getCheckpointingInterval(); - if(checkpointInterval != -1) { - if (checkpointInterval < 1) { - throw new IllegalArgumentException("The checkpoint interval must be positive"); - } - this.flinkStreamEnv.enableCheckpointing(checkpointInterval); - } - } + // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink). + // If the value is not -1, then the validity checks are applied. + // By default, checkpointing is disabled. + long checkpointInterval = options.getCheckpointingInterval(); + if(checkpointInterval != -1) { + if (checkpointInterval < 1) { + throw new IllegalArgumentException("The checkpoint interval must be positive"); + } + this.flinkStreamEnv.enableCheckpointing(checkpointInterval); + } + } - private void checkInitializationState() { - if (options.isStreaming() && this.flinkBatchEnv != null) { - throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment."); - } else if (!options.isStreaming() && this.flinkStreamEnv != null) { - throw new IllegalStateException("Attempted to run a Batch Job with a Streaming Execution Environment."); - } - } + private void checkInitializationState() { + if (options.isStreaming() && this.flinkBatchEnv != null) { + throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment."); + } else if (!options.isStreaming() && this.flinkStreamEnv != null) { + throw new IllegalStateException("Attempted to run a Batch Job with a Streaming Execution Environment."); + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java index 2429caca8153..fabbfadb980b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java @@ -31,61 +31,61 @@ */ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOptions, StreamingOptions { - /** - * List of local files to make available to workers. - *

    - * Jars are placed on the worker's classpath. - *

    - * The default value is the list of jars from the main program's classpath. - */ - @Description("Jar-Files to send to all workers and put on the classpath. " + - "The default value is all files from the classpath.") - @JsonIgnore - List getFilesToStage(); - void setFilesToStage(List value); + /** + * List of local files to make available to workers. + *

    + * Jars are placed on the worker's classpath. + *

    + * The default value is the list of jars from the main program's classpath. + */ + @Description("Jar-Files to send to all workers and put on the classpath. " + + "The default value is all files from the classpath.") + @JsonIgnore + List getFilesToStage(); + void setFilesToStage(List value); - /** - * The job name is used to identify jobs running on a Flink cluster. - */ - @Description("Dataflow job name, to uniquely identify active jobs. " - + "Defaults to using the ApplicationName-UserName-Date.") - @Default.InstanceFactory(DataflowPipelineOptions.JobNameFactory.class) - String getJobName(); - void setJobName(String value); + /** + * The job name is used to identify jobs running on a Flink cluster. + */ + @Description("Dataflow job name, to uniquely identify active jobs. " + + "Defaults to using the ApplicationName-UserName-Date.") + @Default.InstanceFactory(DataflowPipelineOptions.JobNameFactory.class) + String getJobName(); + void setJobName(String value); - /** - * The url of the Flink JobManager on which to execute pipelines. This can either be - * the the address of a cluster JobManager, in the form "host:port" or one of the special - * Strings "[local]", "[collection]" or "[auto]". "[local]" will start a local Flink - * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while - * "[auto]" will let the system decide where to execute the pipeline based on the environment. - */ - @Description("Address of the Flink Master where the Pipeline should be executed. Can" + - " either be of the form \"host:port\" or one of the special values [local], " + - "[collection] or [auto].") - String getFlinkMaster(); - void setFlinkMaster(String value); + /** + * The url of the Flink JobManager on which to execute pipelines. This can either be + * the the address of a cluster JobManager, in the form "host:port" or one of the special + * Strings "[local]", "[collection]" or "[auto]". "[local]" will start a local Flink + * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while + * "[auto]" will let the system decide where to execute the pipeline based on the environment. + */ + @Description("Address of the Flink Master where the Pipeline should be executed. Can" + + " either be of the form \"host:port\" or one of the special values [local], " + + "[collection] or [auto].") + String getFlinkMaster(); + void setFlinkMaster(String value); - @Description("The degree of parallelism to be used when distributing operations onto workers.") - @Default.Integer(-1) - Integer getParallelism(); - void setParallelism(Integer value); + @Description("The degree of parallelism to be used when distributing operations onto workers.") + @Default.Integer(-1) + Integer getParallelism(); + void setParallelism(Integer value); - @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " + - "fault tolerance).") - @Default.Long(-1L) - Long getCheckpointingInterval(); - void setCheckpointingInterval(Long interval); + @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " + + "fault tolerance).") + @Default.Long(-1L) + Long getCheckpointingInterval(); + void setCheckpointingInterval(Long interval); - @Description("Sets the number of times that failed tasks are re-executed. " + - "A value of zero effectively disables fault tolerance. A value of -1 indicates " + - "that the system default value (as defined in the configuration) should be used.") - @Default.Integer(-1) - Integer getNumberOfExecutionRetries(); - void setNumberOfExecutionRetries(Integer retries); + @Description("Sets the number of times that failed tasks are re-executed. " + + "A value of zero effectively disables fault tolerance. A value of -1 indicates " + + "that the system default value (as defined in the configuration) should be used.") + @Default.Integer(-1) + Integer getNumberOfExecutionRetries(); + void setNumberOfExecutionRetries(Integer retries); - @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.") - @Default.Long(-1L) - Long getExecutionRetryDelay(); - void setExecutionRetryDelay(Long delay); + @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.") + @Default.Long(-1L) + Long getExecutionRetryDelay(); + void setExecutionRetryDelay(Long delay); } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java index 7ea837016aca..742a31643239 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java @@ -47,158 +47,158 @@ */ public class FlinkPipelineRunner extends PipelineRunner { - private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineRunner.class); - - /** - * Provided options. - */ - private final FlinkPipelineOptions options; - - private final FlinkPipelineExecutionEnvironment flinkJobEnv; - - /** - * Construct a runner from the provided options. - * - * @param options Properties which configure the runner. - * @return The newly created runner. - */ - public static FlinkPipelineRunner fromOptions(PipelineOptions options) { - FlinkPipelineOptions flinkOptions = - PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options); - ArrayList missing = new ArrayList<>(); - - if (flinkOptions.getAppName() == null) { - missing.add("appName"); - } - if (missing.size() > 0) { - throw new IllegalArgumentException( - "Missing required values: " + Joiner.on(',').join(missing)); - } - - if (flinkOptions.getFilesToStage() == null) { - flinkOptions.setFilesToStage(detectClassPathResourcesToStage( - DataflowPipelineRunner.class.getClassLoader())); - LOG.info("PipelineOptions.filesToStage was not specified. " - + "Defaulting to files from the classpath: will stage {} files. " - + "Enable logging at DEBUG level to see which files will be staged.", - flinkOptions.getFilesToStage().size()); - LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage()); - } - - // Verify jobName according to service requirements. - String jobName = flinkOptions.getJobName().toLowerCase(); - Preconditions.checkArgument(jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"), "JobName invalid; " + - "the name must consist of only the characters " + "[-a-z0-9], starting with a letter " + - "and ending with a letter " + "or number"); - Preconditions.checkArgument(jobName.length() <= 40, - "JobName too long; must be no more than 40 characters in length"); - - // Set Flink Master to [auto] if no option was specified. - if (flinkOptions.getFlinkMaster() == null) { - flinkOptions.setFlinkMaster("[auto]"); - } - - return new FlinkPipelineRunner(flinkOptions); - } - - private FlinkPipelineRunner(FlinkPipelineOptions options) { - this.options = options; - this.flinkJobEnv = new FlinkPipelineExecutionEnvironment(options); - } - - @Override - public FlinkRunnerResult run(Pipeline pipeline) { - LOG.info("Executing pipeline using FlinkPipelineRunner."); - - LOG.info("Translating pipeline to Flink program."); - - this.flinkJobEnv.translate(pipeline); - - LOG.info("Starting execution of Flink program."); - - JobExecutionResult result; - try { - result = this.flinkJobEnv.executePipeline(); - } catch (Exception e) { - LOG.error("Pipeline execution failed", e); - throw new RuntimeException("Pipeline execution failed", e); - } - - LOG.info("Execution finished in {} msecs", result.getNetRuntime()); - - Map accumulators = result.getAllAccumulatorResults(); - if (accumulators != null && !accumulators.isEmpty()) { - LOG.info("Final aggregator values:"); - - for (Map.Entry entry : result.getAllAccumulatorResults().entrySet()) { - LOG.info("{} : {}", entry.getKey(), entry.getValue()); - } - } - - return new FlinkRunnerResult(accumulators, result.getNetRuntime()); - } - - /** - * For testing. - */ - public FlinkPipelineOptions getPipelineOptions() { - return options; - } - - /** - * Constructs a runner with default properties for testing. - * - * @return The newly created runner. - */ - public static FlinkPipelineRunner createForTest(boolean streaming) { - FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class); - // we use [auto] for testing since this will make it pick up the Testing - // ExecutionEnvironment - options.setFlinkMaster("[auto]"); - options.setStreaming(streaming); - return new FlinkPipelineRunner(options); - } - - @Override - public Output apply( - PTransform transform, Input input) { - return super.apply(transform, input); - } - - ///////////////////////////////////////////////////////////////////////////// - - @Override - public String toString() { - return "DataflowPipelineRunner#" + hashCode(); - } - - /** - * Attempts to detect all the resources the class loader has access to. This does not recurse - * to class loader parents stopping it from pulling in resources from the system class loader. - * - * @param classLoader The URLClassLoader to use to detect resources to stage. - * @return A list of absolute paths to the resources the class loader uses. - * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one - * of the resources the class loader exposes is not a file resource. - */ - protected static List detectClassPathResourcesToStage(ClassLoader classLoader) { - if (!(classLoader instanceof URLClassLoader)) { - String message = String.format("Unable to use ClassLoader to detect classpath elements. " - + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader); - LOG.error(message); - throw new IllegalArgumentException(message); - } - - List files = new ArrayList<>(); - for (URL url : ((URLClassLoader) classLoader).getURLs()) { - try { - files.add(new File(url.toURI()).getAbsolutePath()); - } catch (IllegalArgumentException | URISyntaxException e) { - String message = String.format("Unable to convert url (%s) to file.", url); - LOG.error(message); - throw new IllegalArgumentException(message, e); - } - } - return files; - } + private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineRunner.class); + + /** + * Provided options. + */ + private final FlinkPipelineOptions options; + + private final FlinkPipelineExecutionEnvironment flinkJobEnv; + + /** + * Construct a runner from the provided options. + * + * @param options Properties which configure the runner. + * @return The newly created runner. + */ + public static FlinkPipelineRunner fromOptions(PipelineOptions options) { + FlinkPipelineOptions flinkOptions = + PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options); + ArrayList missing = new ArrayList<>(); + + if (flinkOptions.getAppName() == null) { + missing.add("appName"); + } + if (missing.size() > 0) { + throw new IllegalArgumentException( + "Missing required values: " + Joiner.on(',').join(missing)); + } + + if (flinkOptions.getFilesToStage() == null) { + flinkOptions.setFilesToStage(detectClassPathResourcesToStage( + DataflowPipelineRunner.class.getClassLoader())); + LOG.info("PipelineOptions.filesToStage was not specified. " + + "Defaulting to files from the classpath: will stage {} files. " + + "Enable logging at DEBUG level to see which files will be staged.", + flinkOptions.getFilesToStage().size()); + LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage()); + } + + // Verify jobName according to service requirements. + String jobName = flinkOptions.getJobName().toLowerCase(); + Preconditions.checkArgument(jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"), "JobName invalid; " + + "the name must consist of only the characters " + "[-a-z0-9], starting with a letter " + + "and ending with a letter " + "or number"); + Preconditions.checkArgument(jobName.length() <= 40, + "JobName too long; must be no more than 40 characters in length"); + + // Set Flink Master to [auto] if no option was specified. + if (flinkOptions.getFlinkMaster() == null) { + flinkOptions.setFlinkMaster("[auto]"); + } + + return new FlinkPipelineRunner(flinkOptions); + } + + private FlinkPipelineRunner(FlinkPipelineOptions options) { + this.options = options; + this.flinkJobEnv = new FlinkPipelineExecutionEnvironment(options); + } + + @Override + public FlinkRunnerResult run(Pipeline pipeline) { + LOG.info("Executing pipeline using FlinkPipelineRunner."); + + LOG.info("Translating pipeline to Flink program."); + + this.flinkJobEnv.translate(pipeline); + + LOG.info("Starting execution of Flink program."); + + JobExecutionResult result; + try { + result = this.flinkJobEnv.executePipeline(); + } catch (Exception e) { + LOG.error("Pipeline execution failed", e); + throw new RuntimeException("Pipeline execution failed", e); + } + + LOG.info("Execution finished in {} msecs", result.getNetRuntime()); + + Map accumulators = result.getAllAccumulatorResults(); + if (accumulators != null && !accumulators.isEmpty()) { + LOG.info("Final aggregator values:"); + + for (Map.Entry entry : result.getAllAccumulatorResults().entrySet()) { + LOG.info("{} : {}", entry.getKey(), entry.getValue()); + } + } + + return new FlinkRunnerResult(accumulators, result.getNetRuntime()); + } + + /** + * For testing. + */ + public FlinkPipelineOptions getPipelineOptions() { + return options; + } + + /** + * Constructs a runner with default properties for testing. + * + * @return The newly created runner. + */ + public static FlinkPipelineRunner createForTest(boolean streaming) { + FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class); + // we use [auto] for testing since this will make it pick up the Testing + // ExecutionEnvironment + options.setFlinkMaster("[auto]"); + options.setStreaming(streaming); + return new FlinkPipelineRunner(options); + } + + @Override + public Output apply( + PTransform transform, Input input) { + return super.apply(transform, input); + } + + ///////////////////////////////////////////////////////////////////////////// + + @Override + public String toString() { + return "DataflowPipelineRunner#" + hashCode(); + } + + /** + * Attempts to detect all the resources the class loader has access to. This does not recurse + * to class loader parents stopping it from pulling in resources from the system class loader. + * + * @param classLoader The URLClassLoader to use to detect resources to stage. + * @return A list of absolute paths to the resources the class loader uses. + * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one + * of the resources the class loader exposes is not a file resource. + */ + protected static List detectClassPathResourcesToStage(ClassLoader classLoader) { + if (!(classLoader instanceof URLClassLoader)) { + String message = String.format("Unable to use ClassLoader to detect classpath elements. " + + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader); + LOG.error(message); + throw new IllegalArgumentException(message); + } + + List files = new ArrayList<>(); + for (URL url : ((URLClassLoader) classLoader).getURLs()) { + try { + files.add(new File(url.toURI()).getAbsolutePath()); + } catch (IllegalArgumentException | URISyntaxException e) { + String message = String.format("Unable to convert url (%s) to file.", url); + LOG.error(message); + throw new IllegalArgumentException(message, e); + } + } + return files; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java index 59b8b63709ed..dfbaf66f41ad 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java @@ -29,38 +29,38 @@ * {@link com.google.cloud.dataflow.sdk.transforms.Aggregator}s. */ public class FlinkRunnerResult implements PipelineResult { - - private final Map aggregators; - - private final long runtime; - - public FlinkRunnerResult(Map aggregators, long runtime) { - this.aggregators = (aggregators == null || aggregators.isEmpty()) ? - Collections.emptyMap() : - Collections.unmodifiableMap(aggregators); - - this.runtime = runtime; - } + + private final Map aggregators; + + private final long runtime; + + public FlinkRunnerResult(Map aggregators, long runtime) { + this.aggregators = (aggregators == null || aggregators.isEmpty()) ? + Collections.emptyMap() : + Collections.unmodifiableMap(aggregators); + + this.runtime = runtime; + } - @Override - public State getState() { - return null; - } + @Override + public State getState() { + return null; + } - @Override - public AggregatorValues getAggregatorValues(final Aggregator aggregator) throws AggregatorRetrievalException { - // TODO provide a list of all accumulator step values - Object value = aggregators.get(aggregator.getName()); - if (value != null) { - return new AggregatorValues() { - @Override - public Map getValuesAtSteps() { - return (Map) aggregators; - } - }; - } else { - throw new AggregatorRetrievalException("Accumulator results not found.", - new RuntimeException("Accumulator does not exist.")); - } - } + @Override + public AggregatorValues getAggregatorValues(final Aggregator aggregator) throws AggregatorRetrievalException { + // TODO provide a list of all accumulator step values + Object value = aggregators.get(aggregator.getName()); + if (value != null) { + return new AggregatorValues() { + @Override + public Map getValuesAtSteps() { + return (Map) aggregators; + } + }; + } else { + throw new AggregatorRetrievalException("Accumulator results not found.", + new RuntimeException("Accumulator does not exist.")); + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java index 86a269583f2f..8accae7dbe16 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java @@ -89,364 +89,364 @@ * {@code --input}. */ public class TFIDF { - /** - * Options supported by {@link TFIDF}. - *

    - * Inherits standard configuration options. - */ - private interface Options extends PipelineOptions, FlinkPipelineOptions { - @Description("Path to the directory or GCS prefix containing files to read from") - @Default.String("gs://dataflow-samples/shakespeare/") - String getInput(); - void setInput(String value); - - @Description("Prefix of output URI to write to") - @Validation.Required - String getOutput(); - void setOutput(String value); - } - - /** - * Lists documents contained beneath the {@code options.input} prefix/directory. - */ - public static Set listInputDocuments(Options options) - throws URISyntaxException, IOException { - URI baseUri = new URI(options.getInput()); - - // List all documents in the directory or GCS prefix. - URI absoluteUri; - if (baseUri.getScheme() != null) { - absoluteUri = baseUri; - } else { - absoluteUri = new URI( - "file", - baseUri.getAuthority(), - baseUri.getPath(), - baseUri.getQuery(), - baseUri.getFragment()); - } - - Set uris = new HashSet<>(); - if (absoluteUri.getScheme().equals("file")) { - File directory = new File(absoluteUri); - for (String entry : directory.list()) { - File path = new File(directory, entry); - uris.add(path.toURI()); - } - } else if (absoluteUri.getScheme().equals("gs")) { - GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil(); - URI gcsUriGlob = new URI( - absoluteUri.getScheme(), - absoluteUri.getAuthority(), - absoluteUri.getPath() + "*", - absoluteUri.getQuery(), - absoluteUri.getFragment()); - for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) { - uris.add(entry.toUri()); - } - } - - return uris; - } - - /** - * Reads the documents at the provided uris and returns all lines - * from the documents tagged with which document they are from. - */ - public static class ReadDocuments - extends PTransform>> { - private static final long serialVersionUID = 0; - - private Iterable uris; - - public ReadDocuments(Iterable uris) { - this.uris = uris; - } - - @Override - public Coder getDefaultOutputCoder() { - return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of()); - } - - @Override - public PCollection> apply(PInput input) { - Pipeline pipeline = input.getPipeline(); - - // Create one TextIO.Read transform for each document - // and add its output to a PCollectionList - PCollectionList> urisToLines = - PCollectionList.empty(pipeline); - - // TextIO.Read supports: - // - file: URIs and paths locally - // - gs: URIs on the service - for (final URI uri : uris) { - String uriString; - if (uri.getScheme().equals("file")) { - uriString = new File(uri).getPath(); - } else { - uriString = uri.toString(); - } - - PCollection> oneUriToLines = pipeline - .apply(TextIO.Read.from(uriString) - .named("TextIO.Read(" + uriString + ")")) - .apply("WithKeys(" + uriString + ")", WithKeys.of(uri)); - - urisToLines = urisToLines.and(oneUriToLines); - } - - return urisToLines.apply(Flatten.>pCollections()); - } - } - - /** - * A transform containing a basic TF-IDF pipeline. The input consists of KV objects - * where the key is the document's URI and the value is a piece - * of the document's content. The output is mapping from terms to - * scores for each document URI. - */ - public static class ComputeTfIdf - extends PTransform>, PCollection>>> { - private static final long serialVersionUID = 0; - - public ComputeTfIdf() { } - - @Override - public PCollection>> apply( - PCollection> uriToContent) { - - // Compute the total number of documents, and - // prepare this singleton PCollectionView for - // use as a side input. - final PCollectionView totalDocuments = - uriToContent - .apply("GetURIs", Keys.create()) - .apply("RemoveDuplicateDocs", RemoveDuplicates.create()) - .apply(Count.globally()) - .apply(View.asSingleton()); - - // Create a collection of pairs mapping a URI to each - // of the words in the document associated with that that URI. - PCollection> uriToWords = uriToContent - .apply(ParDo.named("SplitWords").of( - new DoFn, KV>() { - private static final long serialVersionUID = 0; - - @Override - public void processElement(ProcessContext c) { - URI uri = c.element().getKey(); - String line = c.element().getValue(); - for (String word : line.split("\\W+")) { - // Log INFO messages when the word “love” is found. - if (word.toLowerCase().equals("love")) { - LOG.info("Found {}", word.toLowerCase()); - } - - if (!word.isEmpty()) { - c.output(KV.of(uri, word.toLowerCase())); - } - } - } - })); - - // Compute a mapping from each word to the total - // number of documents in which it appears. - PCollection> wordToDocCount = uriToWords - .apply("RemoveDuplicateWords", RemoveDuplicates.>create()) - .apply(Values.create()) - .apply("CountDocs", Count.perElement()); - - // Compute a mapping from each URI to the total - // number of words in the document associated with that URI. - PCollection> uriToWordTotal = uriToWords - .apply("GetURIs2", Keys.create()) - .apply("CountWords", Count.perElement()); - - // Count, for each (URI, word) pair, the number of - // occurrences of that word in the document associated - // with the URI. - PCollection, Long>> uriAndWordToCount = uriToWords - .apply("CountWordDocPairs", Count.>perElement()); - - // Adjust the above collection to a mapping from - // (URI, word) pairs to counts into an isomorphic mapping - // from URI to (word, count) pairs, to prepare for a join - // by the URI key. - PCollection>> uriToWordAndCount = uriAndWordToCount - .apply(ParDo.named("ShiftKeys").of( - new DoFn, Long>, KV>>() { - private static final long serialVersionUID = 0; - - @Override - public void processElement(ProcessContext c) { - URI uri = c.element().getKey().getKey(); - String word = c.element().getKey().getValue(); - Long occurrences = c.element().getValue(); - c.output(KV.of(uri, KV.of(word, occurrences))); - } - })); - - // Prepare to join the mapping of URI to (word, count) pairs with - // the mapping of URI to total word counts, by associating - // each of the input PCollection> with - // a tuple tag. Each input must have the same key type, URI - // in this case. The type parameter of the tuple tag matches - // the types of the values for each collection. - final TupleTag wordTotalsTag = new TupleTag<>(); - final TupleTag> wordCountsTag = new TupleTag<>(); - KeyedPCollectionTuple coGbkInput = KeyedPCollectionTuple - .of(wordTotalsTag, uriToWordTotal) - .and(wordCountsTag, uriToWordAndCount); - - // Perform a CoGroupByKey (a sort of pre-join) on the prepared - // inputs. This yields a mapping from URI to a CoGbkResult - // (CoGroupByKey Result). The CoGbkResult is a mapping - // from the above tuple tags to the values in each input - // associated with a particular URI. In this case, each - // KV group a URI with the total number of - // words in that document as well as all the (word, count) - // pairs for particular words. - PCollection> uriToWordAndCountAndTotal = coGbkInput - .apply("CoGroupByUri", CoGroupByKey.create()); - - // Compute a mapping from each word to a (URI, term frequency) - // pair for each URI. A word's term frequency for a document - // is simply the number of times that word occurs in the document - // divided by the total number of words in the document. - PCollection>> wordToUriAndTf = uriToWordAndCountAndTotal - .apply(ParDo.named("ComputeTermFrequencies").of( - new DoFn, KV>>() { - private static final long serialVersionUID = 0; - - @Override - public void processElement(ProcessContext c) { - URI uri = c.element().getKey(); - Long wordTotal = c.element().getValue().getOnly(wordTotalsTag); - - for (KV wordAndCount - : c.element().getValue().getAll(wordCountsTag)) { - String word = wordAndCount.getKey(); - Long wordCount = wordAndCount.getValue(); - Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue(); - c.output(KV.of(word, KV.of(uri, termFrequency))); - } - } - })); - - // Compute a mapping from each word to its document frequency. - // A word's document frequency in a corpus is the number of - // documents in which the word appears divided by the total - // number of documents in the corpus. Note how the total number of - // documents is passed as a side input; the same value is - // presented to each invocation of the DoFn. - PCollection> wordToDf = wordToDocCount - .apply(ParDo - .named("ComputeDocFrequencies") - .withSideInputs(totalDocuments) - .of(new DoFn, KV>() { - private static final long serialVersionUID = 0; - - @Override - public void processElement(ProcessContext c) { - String word = c.element().getKey(); - Long documentCount = c.element().getValue(); - Long documentTotal = c.sideInput(totalDocuments); - Double documentFrequency = documentCount.doubleValue() - / documentTotal.doubleValue(); - - c.output(KV.of(word, documentFrequency)); - } - })); - - // Join the term frequency and document frequency - // collections, each keyed on the word. - final TupleTag> tfTag = new TupleTag<>(); - final TupleTag dfTag = new TupleTag<>(); - PCollection> wordToUriAndTfAndDf = KeyedPCollectionTuple - .of(tfTag, wordToUriAndTf) - .and(dfTag, wordToDf) - .apply(CoGroupByKey.create()); - - // Compute a mapping from each word to a (URI, TF-IDF) score - // for each URI. There are a variety of definitions of TF-IDF - // ("term frequency - inverse document frequency") score; - // here we use a basic version that is the term frequency - // divided by the log of the document frequency. - - return wordToUriAndTfAndDf - .apply(ParDo.named("ComputeTfIdf").of( - new DoFn, KV>>() { - private static final long serialVersionUID1 = 0; - - @Override - public void processElement(ProcessContext c) { - String word = c.element().getKey(); - Double df = c.element().getValue().getOnly(dfTag); - - for (KV uriAndTf : c.element().getValue().getAll(tfTag)) { - URI uri = uriAndTf.getKey(); - Double tf = uriAndTf.getValue(); - Double tfIdf = tf * Math.log(1 / df); - c.output(KV.of(word, KV.of(uri, tfIdf))); - } - } - })); - } - - // Instantiate Logger. - // It is suggested that the user specify the class name of the containing class - // (in this case ComputeTfIdf). - private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class); - } - - /** - * A {@link PTransform} to write, in CSV format, a mapping from term and URI - * to score. - */ - public static class WriteTfIdf - extends PTransform>>, PDone> { - private static final long serialVersionUID = 0; - - private String output; - - public WriteTfIdf(String output) { - this.output = output; - } - - @Override - public PDone apply(PCollection>> wordToUriAndTfIdf) { - return wordToUriAndTfIdf - .apply(ParDo.named("Format").of(new DoFn>, String>() { - private static final long serialVersionUID = 0; - - @Override - public void processElement(ProcessContext c) { - c.output(String.format("%s,\t%s,\t%f", - c.element().getKey(), - c.element().getValue().getKey(), - c.element().getValue().getValue())); - } - })) - .apply(TextIO.Write - .to(output) - .withSuffix(".csv")); - } - } - - public static void main(String[] args) throws Exception { - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - - options.setRunner(FlinkPipelineRunner.class); - - Pipeline pipeline = Pipeline.create(options); - pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); - - pipeline - .apply(new ReadDocuments(listInputDocuments(options))) - .apply(new ComputeTfIdf()) - .apply(new WriteTfIdf(options.getOutput())); - - pipeline.run(); - } + /** + * Options supported by {@link TFIDF}. + *

    + * Inherits standard configuration options. + */ + private interface Options extends PipelineOptions, FlinkPipelineOptions { + @Description("Path to the directory or GCS prefix containing files to read from") + @Default.String("gs://dataflow-samples/shakespeare/") + String getInput(); + void setInput(String value); + + @Description("Prefix of output URI to write to") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + /** + * Lists documents contained beneath the {@code options.input} prefix/directory. + */ + public static Set listInputDocuments(Options options) + throws URISyntaxException, IOException { + URI baseUri = new URI(options.getInput()); + + // List all documents in the directory or GCS prefix. + URI absoluteUri; + if (baseUri.getScheme() != null) { + absoluteUri = baseUri; + } else { + absoluteUri = new URI( + "file", + baseUri.getAuthority(), + baseUri.getPath(), + baseUri.getQuery(), + baseUri.getFragment()); + } + + Set uris = new HashSet<>(); + if (absoluteUri.getScheme().equals("file")) { + File directory = new File(absoluteUri); + for (String entry : directory.list()) { + File path = new File(directory, entry); + uris.add(path.toURI()); + } + } else if (absoluteUri.getScheme().equals("gs")) { + GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil(); + URI gcsUriGlob = new URI( + absoluteUri.getScheme(), + absoluteUri.getAuthority(), + absoluteUri.getPath() + "*", + absoluteUri.getQuery(), + absoluteUri.getFragment()); + for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) { + uris.add(entry.toUri()); + } + } + + return uris; + } + + /** + * Reads the documents at the provided uris and returns all lines + * from the documents tagged with which document they are from. + */ + public static class ReadDocuments + extends PTransform>> { + private static final long serialVersionUID = 0; + + private Iterable uris; + + public ReadDocuments(Iterable uris) { + this.uris = uris; + } + + @Override + public Coder getDefaultOutputCoder() { + return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of()); + } + + @Override + public PCollection> apply(PInput input) { + Pipeline pipeline = input.getPipeline(); + + // Create one TextIO.Read transform for each document + // and add its output to a PCollectionList + PCollectionList> urisToLines = + PCollectionList.empty(pipeline); + + // TextIO.Read supports: + // - file: URIs and paths locally + // - gs: URIs on the service + for (final URI uri : uris) { + String uriString; + if (uri.getScheme().equals("file")) { + uriString = new File(uri).getPath(); + } else { + uriString = uri.toString(); + } + + PCollection> oneUriToLines = pipeline + .apply(TextIO.Read.from(uriString) + .named("TextIO.Read(" + uriString + ")")) + .apply("WithKeys(" + uriString + ")", WithKeys.of(uri)); + + urisToLines = urisToLines.and(oneUriToLines); + } + + return urisToLines.apply(Flatten.>pCollections()); + } + } + + /** + * A transform containing a basic TF-IDF pipeline. The input consists of KV objects + * where the key is the document's URI and the value is a piece + * of the document's content. The output is mapping from terms to + * scores for each document URI. + */ + public static class ComputeTfIdf + extends PTransform>, PCollection>>> { + private static final long serialVersionUID = 0; + + public ComputeTfIdf() { } + + @Override + public PCollection>> apply( + PCollection> uriToContent) { + + // Compute the total number of documents, and + // prepare this singleton PCollectionView for + // use as a side input. + final PCollectionView totalDocuments = + uriToContent + .apply("GetURIs", Keys.create()) + .apply("RemoveDuplicateDocs", RemoveDuplicates.create()) + .apply(Count.globally()) + .apply(View.asSingleton()); + + // Create a collection of pairs mapping a URI to each + // of the words in the document associated with that that URI. + PCollection> uriToWords = uriToContent + .apply(ParDo.named("SplitWords").of( + new DoFn, KV>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey(); + String line = c.element().getValue(); + for (String word : line.split("\\W+")) { + // Log INFO messages when the word “love” is found. + if (word.toLowerCase().equals("love")) { + LOG.info("Found {}", word.toLowerCase()); + } + + if (!word.isEmpty()) { + c.output(KV.of(uri, word.toLowerCase())); + } + } + } + })); + + // Compute a mapping from each word to the total + // number of documents in which it appears. + PCollection> wordToDocCount = uriToWords + .apply("RemoveDuplicateWords", RemoveDuplicates.>create()) + .apply(Values.create()) + .apply("CountDocs", Count.perElement()); + + // Compute a mapping from each URI to the total + // number of words in the document associated with that URI. + PCollection> uriToWordTotal = uriToWords + .apply("GetURIs2", Keys.create()) + .apply("CountWords", Count.perElement()); + + // Count, for each (URI, word) pair, the number of + // occurrences of that word in the document associated + // with the URI. + PCollection, Long>> uriAndWordToCount = uriToWords + .apply("CountWordDocPairs", Count.>perElement()); + + // Adjust the above collection to a mapping from + // (URI, word) pairs to counts into an isomorphic mapping + // from URI to (word, count) pairs, to prepare for a join + // by the URI key. + PCollection>> uriToWordAndCount = uriAndWordToCount + .apply(ParDo.named("ShiftKeys").of( + new DoFn, Long>, KV>>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey().getKey(); + String word = c.element().getKey().getValue(); + Long occurrences = c.element().getValue(); + c.output(KV.of(uri, KV.of(word, occurrences))); + } + })); + + // Prepare to join the mapping of URI to (word, count) pairs with + // the mapping of URI to total word counts, by associating + // each of the input PCollection> with + // a tuple tag. Each input must have the same key type, URI + // in this case. The type parameter of the tuple tag matches + // the types of the values for each collection. + final TupleTag wordTotalsTag = new TupleTag<>(); + final TupleTag> wordCountsTag = new TupleTag<>(); + KeyedPCollectionTuple coGbkInput = KeyedPCollectionTuple + .of(wordTotalsTag, uriToWordTotal) + .and(wordCountsTag, uriToWordAndCount); + + // Perform a CoGroupByKey (a sort of pre-join) on the prepared + // inputs. This yields a mapping from URI to a CoGbkResult + // (CoGroupByKey Result). The CoGbkResult is a mapping + // from the above tuple tags to the values in each input + // associated with a particular URI. In this case, each + // KV group a URI with the total number of + // words in that document as well as all the (word, count) + // pairs for particular words. + PCollection> uriToWordAndCountAndTotal = coGbkInput + .apply("CoGroupByUri", CoGroupByKey.create()); + + // Compute a mapping from each word to a (URI, term frequency) + // pair for each URI. A word's term frequency for a document + // is simply the number of times that word occurs in the document + // divided by the total number of words in the document. + PCollection>> wordToUriAndTf = uriToWordAndCountAndTotal + .apply(ParDo.named("ComputeTermFrequencies").of( + new DoFn, KV>>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + URI uri = c.element().getKey(); + Long wordTotal = c.element().getValue().getOnly(wordTotalsTag); + + for (KV wordAndCount + : c.element().getValue().getAll(wordCountsTag)) { + String word = wordAndCount.getKey(); + Long wordCount = wordAndCount.getValue(); + Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue(); + c.output(KV.of(word, KV.of(uri, termFrequency))); + } + } + })); + + // Compute a mapping from each word to its document frequency. + // A word's document frequency in a corpus is the number of + // documents in which the word appears divided by the total + // number of documents in the corpus. Note how the total number of + // documents is passed as a side input; the same value is + // presented to each invocation of the DoFn. + PCollection> wordToDf = wordToDocCount + .apply(ParDo + .named("ComputeDocFrequencies") + .withSideInputs(totalDocuments) + .of(new DoFn, KV>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + String word = c.element().getKey(); + Long documentCount = c.element().getValue(); + Long documentTotal = c.sideInput(totalDocuments); + Double documentFrequency = documentCount.doubleValue() + / documentTotal.doubleValue(); + + c.output(KV.of(word, documentFrequency)); + } + })); + + // Join the term frequency and document frequency + // collections, each keyed on the word. + final TupleTag> tfTag = new TupleTag<>(); + final TupleTag dfTag = new TupleTag<>(); + PCollection> wordToUriAndTfAndDf = KeyedPCollectionTuple + .of(tfTag, wordToUriAndTf) + .and(dfTag, wordToDf) + .apply(CoGroupByKey.create()); + + // Compute a mapping from each word to a (URI, TF-IDF) score + // for each URI. There are a variety of definitions of TF-IDF + // ("term frequency - inverse document frequency") score; + // here we use a basic version that is the term frequency + // divided by the log of the document frequency. + + return wordToUriAndTfAndDf + .apply(ParDo.named("ComputeTfIdf").of( + new DoFn, KV>>() { + private static final long serialVersionUID1 = 0; + + @Override + public void processElement(ProcessContext c) { + String word = c.element().getKey(); + Double df = c.element().getValue().getOnly(dfTag); + + for (KV uriAndTf : c.element().getValue().getAll(tfTag)) { + URI uri = uriAndTf.getKey(); + Double tf = uriAndTf.getValue(); + Double tfIdf = tf * Math.log(1 / df); + c.output(KV.of(word, KV.of(uri, tfIdf))); + } + } + })); + } + + // Instantiate Logger. + // It is suggested that the user specify the class name of the containing class + // (in this case ComputeTfIdf). + private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class); + } + + /** + * A {@link PTransform} to write, in CSV format, a mapping from term and URI + * to score. + */ + public static class WriteTfIdf + extends PTransform>>, PDone> { + private static final long serialVersionUID = 0; + + private String output; + + public WriteTfIdf(String output) { + this.output = output; + } + + @Override + public PDone apply(PCollection>> wordToUriAndTfIdf) { + return wordToUriAndTfIdf + .apply(ParDo.named("Format").of(new DoFn>, String>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + c.output(String.format("%s,\t%s,\t%f", + c.element().getKey(), + c.element().getValue().getKey(), + c.element().getValue().getValue())); + } + })) + .apply(TextIO.Write + .to(output) + .withSuffix(".csv")); + } + } + + public static void main(String[] args) throws Exception { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + options.setRunner(FlinkPipelineRunner.class); + + Pipeline pipeline = Pipeline.create(options); + pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); + + pipeline + .apply(new ReadDocuments(listInputDocuments(options))) + .apply(new ComputeTfIdf()) + .apply(new WriteTfIdf(options.getOutput())); + + pipeline.run(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java index e737fe8fdaf9..4f721b4c10ae 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java @@ -29,83 +29,83 @@ public class WordCount { - public static class ExtractWordsFn extends DoFn { - private final Aggregator emptyLines = - createAggregator("emptyLines", new Sum.SumLongFn()); - - @Override - public void processElement(ProcessContext c) { - if (c.element().trim().isEmpty()) { - emptyLines.addValue(1L); - } - - // Split the line into words. - String[] words = c.element().split("[^a-zA-Z']+"); - - // Output each word encountered into the output PCollection. - for (String word : words) { - if (!word.isEmpty()) { - c.output(word); - } - } - } - } - - public static class CountWords extends PTransform, + public static class ExtractWordsFn extends DoFn { + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + if (c.element().trim().isEmpty()) { + emptyLines.addValue(1L); + } + + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + public static class CountWords extends PTransform, PCollection>> { - @Override - public PCollection> apply(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. - PCollection> wordCounts = - words.apply(Count.perElement()); - - return wordCounts; - } - } - - /** A SimpleFunction that converts a Word and Count into a printable string. */ - public static class FormatAsTextFn extends SimpleFunction, String> { - @Override - public String apply(KV input) { - return input.getKey() + ": " + input.getValue(); - } - } - - /** - * Options supported by {@link WordCount}. - *

    - * Inherits standard configuration options. - */ - public interface Options extends PipelineOptions, FlinkPipelineOptions { - @Description("Path of the file to read from") - @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") - String getInput(); - void setInput(String value); - - @Description("Path of the file to write to") - String getOutput(); - void setOutput(String value); - } - - public static void main(String[] args) { - - Options options = PipelineOptionsFactory.fromArgs(args).withValidation() - .as(Options.class); - options.setRunner(FlinkPipelineRunner.class); - - Pipeline p = Pipeline.create(options); - - p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) - .apply(new CountWords()) - .apply(MapElements.via(new FormatAsTextFn())) - .apply(TextIO.Write.named("WriteCounts").to(options.getOutput())); - - p.run(); - } + @Override + public PCollection> apply(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. + PCollection> wordCounts = + words.apply(Count.perElement()); + + return wordCounts; + } + } + + /** A SimpleFunction that converts a Word and Count into a printable string. */ + public static class FormatAsTextFn extends SimpleFunction, String> { + @Override + public String apply(KV input) { + return input.getKey() + ": " + input.getValue(); + } + } + + /** + * Options supported by {@link WordCount}. + *

    + * Inherits standard configuration options. + */ + public interface Options extends PipelineOptions, FlinkPipelineOptions { + @Description("Path of the file to read from") + @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + String getInput(); + void setInput(String value); + + @Description("Path of the file to write to") + String getOutput(); + void setOutput(String value); + } + + public static void main(String[] args) { + + Options options = PipelineOptionsFactory.fromArgs(args).withValidation() + .as(Options.class); + options.setRunner(FlinkPipelineRunner.class); + + Pipeline p = Pipeline.create(options); + + p.apply(TextIO.Read.named("ReadLines").from(options.getInput())) + .apply(new CountWords()) + .apply(MapElements.via(new FormatAsTextFn())) + .apply(TextIO.Write.named("WriteCounts").to(options.getOutput())); + + p.run(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java index 60f6788561cd..a6e1e37fbe04 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java @@ -50,109 +50,109 @@ * */ public class JoinExamples { - static PCollection joinEvents(PCollection streamA, - PCollection streamB) throws Exception { - - final TupleTag firstInfoTag = new TupleTag<>(); - final TupleTag secondInfoTag = new TupleTag<>(); - - // transform both input collections to tuple collections, where the keys are country - // codes in both cases. - PCollection> firstInfo = streamA.apply( - ParDo.of(new ExtractEventDataFn())); - PCollection> secondInfo = streamB.apply( - ParDo.of(new ExtractEventDataFn())); - - // country code 'key' -> CGBKR (, ) - PCollection> kvpCollection = KeyedPCollectionTuple - .of(firstInfoTag, firstInfo) - .and(secondInfoTag, secondInfo) - .apply(CoGroupByKey.create()); - - // Process the CoGbkResult elements generated by the CoGroupByKey transform. - // country code 'key' -> string of , - PCollection> finalResultCollection = - kvpCollection.apply(ParDo.named("Process").of( - new DoFn, KV>() { - private static final long serialVersionUID = 0; - - @Override - public void processElement(ProcessContext c) { - KV e = c.element(); - String key = e.getKey(); - - String defaultA = "NO_VALUE"; - - // the following getOnly is a bit tricky because it expects to have - // EXACTLY ONE value in the corresponding stream and for the corresponding key. - - String lineA = e.getValue().getOnly(firstInfoTag, defaultA); - for (String lineB : c.element().getValue().getAll(secondInfoTag)) { - // Generate a string that combines information from both collection values - c.output(KV.of(key, "Value A: " + lineA + " - Value B: " + lineB)); - } - } - })); - - return finalResultCollection - .apply(ParDo.named("Format").of(new DoFn, String>() { - private static final long serialVersionUID = 0; - - @Override - public void processElement(ProcessContext c) { - String result = c.element().getKey() + " -> " + c.element().getValue(); - System.out.println(result); - c.output(result); - } - })); - } - - static class ExtractEventDataFn extends DoFn> { - private static final long serialVersionUID = 0; - - @Override - public void processElement(ProcessContext c) { - String line = c.element().toLowerCase(); - String key = line.split("\\s")[0]; - c.output(KV.of(key, line)); - } - } - - private interface Options extends WindowedWordCount.StreamingWordCountOptions { - - } - - public static void main(String[] args) throws Exception { - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - options.setStreaming(true); - options.setCheckpointingInterval(1000L); - options.setNumberOfExecutionRetries(5); - options.setExecutionRetryDelay(3000L); - options.setRunner(FlinkPipelineRunner.class); - - PTransform> readSourceA = - Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("FirstStream"); - PTransform> readSourceB = - Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)).named("SecondStream"); - - WindowFn windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize())); - - Pipeline p = Pipeline.create(options); - - // the following two 'applys' create multiple inputs to our pipeline, one for each - // of our two input sources. - PCollection streamA = p.apply(readSourceA) - .apply(Window.into(windowFn) - .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) - .discardingFiredPanes()); - PCollection streamB = p.apply(readSourceB) - .apply(Window.into(windowFn) - .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) - .discardingFiredPanes()); - - PCollection formattedResults = joinEvents(streamA, streamB); - formattedResults.apply(TextIO.Write.to("./outputJoin.txt")); - p.run(); - } + static PCollection joinEvents(PCollection streamA, + PCollection streamB) throws Exception { + + final TupleTag firstInfoTag = new TupleTag<>(); + final TupleTag secondInfoTag = new TupleTag<>(); + + // transform both input collections to tuple collections, where the keys are country + // codes in both cases. + PCollection> firstInfo = streamA.apply( + ParDo.of(new ExtractEventDataFn())); + PCollection> secondInfo = streamB.apply( + ParDo.of(new ExtractEventDataFn())); + + // country code 'key' -> CGBKR (, ) + PCollection> kvpCollection = KeyedPCollectionTuple + .of(firstInfoTag, firstInfo) + .and(secondInfoTag, secondInfo) + .apply(CoGroupByKey.create()); + + // Process the CoGbkResult elements generated by the CoGroupByKey transform. + // country code 'key' -> string of , + PCollection> finalResultCollection = + kvpCollection.apply(ParDo.named("Process").of( + new DoFn, KV>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + String key = e.getKey(); + + String defaultA = "NO_VALUE"; + + // the following getOnly is a bit tricky because it expects to have + // EXACTLY ONE value in the corresponding stream and for the corresponding key. + + String lineA = e.getValue().getOnly(firstInfoTag, defaultA); + for (String lineB : c.element().getValue().getAll(secondInfoTag)) { + // Generate a string that combines information from both collection values + c.output(KV.of(key, "Value A: " + lineA + " - Value B: " + lineB)); + } + } + })); + + return finalResultCollection + .apply(ParDo.named("Format").of(new DoFn, String>() { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + String result = c.element().getKey() + " -> " + c.element().getValue(); + System.out.println(result); + c.output(result); + } + })); + } + + static class ExtractEventDataFn extends DoFn> { + private static final long serialVersionUID = 0; + + @Override + public void processElement(ProcessContext c) { + String line = c.element().toLowerCase(); + String key = line.split("\\s")[0]; + c.output(KV.of(key, line)); + } + } + + private interface Options extends WindowedWordCount.StreamingWordCountOptions { + + } + + public static void main(String[] args) throws Exception { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + options.setStreaming(true); + options.setCheckpointingInterval(1000L); + options.setNumberOfExecutionRetries(5); + options.setExecutionRetryDelay(3000L); + options.setRunner(FlinkPipelineRunner.class); + + PTransform> readSourceA = + Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("FirstStream"); + PTransform> readSourceB = + Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)).named("SecondStream"); + + WindowFn windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize())); + + Pipeline p = Pipeline.create(options); + + // the following two 'applys' create multiple inputs to our pipeline, one for each + // of our two input sources. + PCollection streamA = p.apply(readSourceA) + .apply(Window.into(windowFn) + .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()); + PCollection streamB = p.apply(readSourceB) + .apply(Window.into(windowFn) + .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()); + + PCollection formattedResults = joinEvents(streamA, streamB); + formattedResults.apply(TextIO.Write.to("./outputJoin.txt")); + p.run(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java index dba2721c5afb..b97c35cfb556 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java @@ -36,106 +36,106 @@ public class KafkaWindowedWordCountExample { - static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from - static final String KAFKA_BROKER = "localhost:9092"; // Default kafka broker to contact - static final String GROUP_ID = "myGroup"; // Default groupId - static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka - - public static class ExtractWordsFn extends DoFn { - private final Aggregator emptyLines = - createAggregator("emptyLines", new Sum.SumLongFn()); - - @Override - public void processElement(ProcessContext c) { - if (c.element().trim().isEmpty()) { - emptyLines.addValue(1L); - } - - // Split the line into words. - String[] words = c.element().split("[^a-zA-Z']+"); - - // Output each word encountered into the output PCollection. - for (String word : words) { - if (!word.isEmpty()) { - c.output(word); - } - } - } - } - - public static class FormatAsStringFn extends DoFn, String> { - @Override - public void processElement(ProcessContext c) { - String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); - System.out.println(row); - c.output(row); - } - } - - public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions { - @Description("The Kafka topic to read from") - @Default.String(KAFKA_TOPIC) - String getKafkaTopic(); - - void setKafkaTopic(String value); - - @Description("The Kafka Broker to read from") - @Default.String(KAFKA_BROKER) - String getBroker(); - - void setBroker(String value); - - @Description("The Zookeeper server to connect to") - @Default.String(ZOOKEEPER) - String getZookeeper(); - - void setZookeeper(String value); - - @Description("The groupId") - @Default.String(GROUP_ID) - String getGroup(); - - void setGroup(String value); - - } - - public static void main(String[] args) { - PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class); - KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class); - options.setJobName("KafkaExample"); - options.setStreaming(true); - options.setCheckpointingInterval(1000L); - options.setNumberOfExecutionRetries(5); - options.setExecutionRetryDelay(3000L); - options.setRunner(FlinkPipelineRunner.class); - - System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() ); - Pipeline pipeline = Pipeline.create(options); - - Properties p = new Properties(); - p.setProperty("zookeeper.connect", options.getZookeeper()); - p.setProperty("bootstrap.servers", options.getBroker()); - p.setProperty("group.id", options.getGroup()); - - // this is the Flink consumer that reads the input to - // the program from a kafka topic. - FlinkKafkaConsumer082 kafkaConsumer = new FlinkKafkaConsumer082<>( - options.getKafkaTopic(), - new SimpleStringSchema(), p); - - PCollection words = pipeline - .apply(Read.from(new UnboundedFlinkSource(options, kafkaConsumer)).named("StreamingWordCount")) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Window.into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize()))) - .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) - .discardingFiredPanes()); - - PCollection> wordCounts = - words.apply(Count.perElement()); - - wordCounts.apply(ParDo.of(new FormatAsStringFn())) - .apply(TextIO.Write.to("./outputKafka.txt")); - - pipeline.run(); - } + static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from + static final String KAFKA_BROKER = "localhost:9092"; // Default kafka broker to contact + static final String GROUP_ID = "myGroup"; // Default groupId + static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka + + public static class ExtractWordsFn extends DoFn { + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + if (c.element().trim().isEmpty()) { + emptyLines.addValue(1L); + } + + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + public static class FormatAsStringFn extends DoFn, String> { + @Override + public void processElement(ProcessContext c) { + String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); + System.out.println(row); + c.output(row); + } + } + + public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions { + @Description("The Kafka topic to read from") + @Default.String(KAFKA_TOPIC) + String getKafkaTopic(); + + void setKafkaTopic(String value); + + @Description("The Kafka Broker to read from") + @Default.String(KAFKA_BROKER) + String getBroker(); + + void setBroker(String value); + + @Description("The Zookeeper server to connect to") + @Default.String(ZOOKEEPER) + String getZookeeper(); + + void setZookeeper(String value); + + @Description("The groupId") + @Default.String(GROUP_ID) + String getGroup(); + + void setGroup(String value); + + } + + public static void main(String[] args) { + PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class); + KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class); + options.setJobName("KafkaExample"); + options.setStreaming(true); + options.setCheckpointingInterval(1000L); + options.setNumberOfExecutionRetries(5); + options.setExecutionRetryDelay(3000L); + options.setRunner(FlinkPipelineRunner.class); + + System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() ); + Pipeline pipeline = Pipeline.create(options); + + Properties p = new Properties(); + p.setProperty("zookeeper.connect", options.getZookeeper()); + p.setProperty("bootstrap.servers", options.getBroker()); + p.setProperty("group.id", options.getGroup()); + + // this is the Flink consumer that reads the input to + // the program from a kafka topic. + FlinkKafkaConsumer082 kafkaConsumer = new FlinkKafkaConsumer082<>( + options.getKafkaTopic(), + new SimpleStringSchema(), p); + + PCollection words = pipeline + .apply(Read.from(new UnboundedFlinkSource(options, kafkaConsumer)).named("StreamingWordCount")) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Window.into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize()))) + .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()); + + PCollection> wordCounts = + words.apply(Count.perElement()); + + wordCounts.apply(ParDo.of(new FormatAsStringFn())) + .apply(TextIO.Write.to("./outputKafka.txt")); + + pipeline.run(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java index 37dc39a6cac5..753cbc3ec71d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java @@ -45,84 +45,84 @@ * */ public class WindowedWordCount { - private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class); - - static final long WINDOW_SIZE = 10; // Default window duration in seconds - static final long SLIDE_SIZE = 5; // Default window slide in seconds - - static class FormatAsStringFn extends DoFn, String> { - @Override - public void processElement(ProcessContext c) { - String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); - c.output(row); - } - } - - static class ExtractWordsFn extends DoFn { - private final Aggregator emptyLines = - createAggregator("emptyLines", new Sum.SumLongFn()); - - @Override - public void processElement(ProcessContext c) { - if (c.element().trim().isEmpty()) { - emptyLines.addValue(1L); - } - - // Split the line into words. - String[] words = c.element().split("[^a-zA-Z']+"); - - // Output each word encountered into the output PCollection. - for (String word : words) { - if (!word.isEmpty()) { - c.output(word); - } - } - } - } - - public interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options { - @Description("Sliding window duration, in seconds") - @Default.Long(WINDOW_SIZE) - Long getWindowSize(); - - void setWindowSize(Long value); - - @Description("Window slide, in seconds") - @Default.Long(SLIDE_SIZE) - Long getSlide(); - - void setSlide(Long value); - } - - public static void main(String[] args) throws IOException { - StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class); - options.setStreaming(true); - options.setWindowSize(10L); - options.setSlide(5L); - options.setCheckpointingInterval(1000L); - options.setNumberOfExecutionRetries(5); - options.setExecutionRetryDelay(3000L); - options.setRunner(FlinkPipelineRunner.class); - - LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() + - " sec. and a slide of " + options.getSlide()); - - Pipeline pipeline = Pipeline.create(options); - - PCollection words = pipeline - .apply(Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("StreamingWordCount")) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Window.into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize())) - .every(Duration.standardSeconds(options.getSlide()))) - .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) - .discardingFiredPanes()); - - PCollection> wordCounts = - words.apply(Count.perElement()); - - wordCounts.apply(ParDo.of(new FormatAsStringFn())) - .apply(TextIO.Write.to("./outputWordCount.txt")); - - pipeline.run(); - } + private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class); + + static final long WINDOW_SIZE = 10; // Default window duration in seconds + static final long SLIDE_SIZE = 5; // Default window slide in seconds + + static class FormatAsStringFn extends DoFn, String> { + @Override + public void processElement(ProcessContext c) { + String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); + c.output(row); + } + } + + static class ExtractWordsFn extends DoFn { + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + if (c.element().trim().isEmpty()) { + emptyLines.addValue(1L); + } + + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + public interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options { + @Description("Sliding window duration, in seconds") + @Default.Long(WINDOW_SIZE) + Long getWindowSize(); + + void setWindowSize(Long value); + + @Description("Window slide, in seconds") + @Default.Long(SLIDE_SIZE) + Long getSlide(); + + void setSlide(Long value); + } + + public static void main(String[] args) throws IOException { + StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class); + options.setStreaming(true); + options.setWindowSize(10L); + options.setSlide(5L); + options.setCheckpointingInterval(1000L); + options.setNumberOfExecutionRetries(5); + options.setExecutionRetryDelay(3000L); + options.setRunner(FlinkPipelineRunner.class); + + LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() + + " sec. and a slide of " + options.getSlide()); + + Pipeline pipeline = Pipeline.create(options); + + PCollection words = pipeline + .apply(Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("StreamingWordCount")) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Window.into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize())) + .every(Duration.standardSeconds(options.getSlide()))) + .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()); + + PCollection> wordCounts = + words.apply(Count.perElement()); + + wordCounts.apply(ParDo.of(new FormatAsStringFn())) + .apply(TextIO.Write.to("./outputWordCount.txt")); + + pipeline.run(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java index 90fb6351bc67..3f3492ccf610 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java @@ -28,53 +28,53 @@ */ public class ConsoleIO { - /** - * A PTransform that writes a PCollection to a standard output. - */ - public static class Write { + /** + * A PTransform that writes a PCollection to a standard output. + */ + public static class Write { - /** - * Returns a ConsoleIO.Write PTransform with a default step name. - */ - public static Bound create() { - return new Bound(); - } + /** + * Returns a ConsoleIO.Write PTransform with a default step name. + */ + public static Bound create() { + return new Bound(); + } - /** - * Returns a ConsoleIO.Write PTransform with the given step name. - */ - public static Bound named(String name) { - return new Bound().named(name); - } + /** + * Returns a ConsoleIO.Write PTransform with the given step name. + */ + public static Bound named(String name) { + return new Bound().named(name); + } - /** - * A PTransform that writes a bounded PCollection to standard output. - */ - public static class Bound extends PTransform, PDone> { - private static final long serialVersionUID = 0; + /** + * A PTransform that writes a bounded PCollection to standard output. + */ + public static class Bound extends PTransform, PDone> { + private static final long serialVersionUID = 0; - Bound() { - super("ConsoleIO.Write"); - } + Bound() { + super("ConsoleIO.Write"); + } - Bound(String name) { - super(name); - } + Bound(String name) { + super(name); + } - /** - * Returns a new ConsoleIO.Write PTransform that's like this one but with the given - * step - * name. Does not modify this object. - */ - public Bound named(String name) { - return new Bound(name); - } + /** + * Returns a new ConsoleIO.Write PTransform that's like this one but with the given + * step + * name. Does not modify this object. + */ + public Bound named(String name) { + return new Bound(name); + } - @Override - public PDone apply(PCollection input) { - return PDone.in(input.getPipeline()); - } - } - } + @Override + public PDone apply(PCollection input) { + return PDone.in(input.getPipeline()); + } + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java index a1e441089117..82b7e970cd40 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java @@ -31,119 +31,119 @@ */ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator { - /** - * The necessary context in the case of a batch job. - */ - private final FlinkBatchTranslationContext batchContext; - - private int depth = 0; - - /** - * Composite transform that we want to translate before proceeding with other transforms. - */ - private PTransform currentCompositeTransform; - - public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) { - this.batchContext = new FlinkBatchTranslationContext(env, options); - } - - // -------------------------------------------------------------------------------------------- - // Pipeline Visitor Methods - // -------------------------------------------------------------------------------------------- - - @Override - public void enterCompositeTransform(TransformTreeNode node) { - System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); - - PTransform transform = node.getTransform(); - if (transform != null && currentCompositeTransform == null) { - - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); - if (translator != null) { - currentCompositeTransform = transform; - if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) { - // we can only optimize CoGroupByKey for input size 2 - currentCompositeTransform = null; - } - } - } - this.depth++; - } - - @Override - public void leaveCompositeTransform(TransformTreeNode node) { - PTransform transform = node.getTransform(); - if (transform != null && currentCompositeTransform == transform) { - - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); - if (translator != null) { - System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); - applyBatchTransform(transform, node, translator); - currentCompositeTransform = null; - } else { - throw new IllegalStateException("Attempted to translate composite transform " + - "but no translator was found: " + currentCompositeTransform); - } - } - this.depth--; - System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node)); - } - - @Override - public void visitTransform(TransformTreeNode node) { - System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node)); - if (currentCompositeTransform != null) { - // ignore it - return; - } - - // get the transformation corresponding to hte node we are - // currently visiting and translate it into its Flink alternative. - - PTransform transform = node.getTransform(); - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); - if (translator == null) { - System.out.println(node.getTransform().getClass()); - throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); - } - applyBatchTransform(transform, node, translator); - } - - @Override - public void visitValue(PValue value, TransformTreeNode producer) { - // do nothing here - } - - private > void applyBatchTransform(PTransform transform, TransformTreeNode node, BatchTransformTranslator translator) { - - @SuppressWarnings("unchecked") - T typedTransform = (T) transform; - - @SuppressWarnings("unchecked") - BatchTransformTranslator typedTranslator = (BatchTransformTranslator) translator; - - // create the applied PTransform on the batchContext - batchContext.setCurrentTransform(AppliedPTransform.of( - node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); - typedTranslator.translateNode(typedTransform, batchContext); - } - - /** - * A translator of a {@link PTransform}. - */ - public interface BatchTransformTranslator { - void translateNode(Type transform, FlinkBatchTranslationContext context); - } - - private static String genSpaces(int n) { - String s = ""; - for (int i = 0; i < n; i++) { - s += "| "; - } - return s; - } - - private static String formatNodeName(TransformTreeNode node) { - return node.toString().split("@")[1] + node.getTransform(); - } + /** + * The necessary context in the case of a batch job. + */ + private final FlinkBatchTranslationContext batchContext; + + private int depth = 0; + + /** + * Composite transform that we want to translate before proceeding with other transforms. + */ + private PTransform currentCompositeTransform; + + public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) { + this.batchContext = new FlinkBatchTranslationContext(env, options); + } + + // -------------------------------------------------------------------------------------------- + // Pipeline Visitor Methods + // -------------------------------------------------------------------------------------------- + + @Override + public void enterCompositeTransform(TransformTreeNode node) { + System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); + + PTransform transform = node.getTransform(); + if (transform != null && currentCompositeTransform == null) { + + BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + if (translator != null) { + currentCompositeTransform = transform; + if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) { + // we can only optimize CoGroupByKey for input size 2 + currentCompositeTransform = null; + } + } + } + this.depth++; + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + if (transform != null && currentCompositeTransform == transform) { + + BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + if (translator != null) { + System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); + applyBatchTransform(transform, node, translator); + currentCompositeTransform = null; + } else { + throw new IllegalStateException("Attempted to translate composite transform " + + "but no translator was found: " + currentCompositeTransform); + } + } + this.depth--; + System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node)); + } + + @Override + public void visitTransform(TransformTreeNode node) { + System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node)); + if (currentCompositeTransform != null) { + // ignore it + return; + } + + // get the transformation corresponding to hte node we are + // currently visiting and translate it into its Flink alternative. + + PTransform transform = node.getTransform(); + BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + if (translator == null) { + System.out.println(node.getTransform().getClass()); + throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); + } + applyBatchTransform(transform, node, translator); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + // do nothing here + } + + private > void applyBatchTransform(PTransform transform, TransformTreeNode node, BatchTransformTranslator translator) { + + @SuppressWarnings("unchecked") + T typedTransform = (T) transform; + + @SuppressWarnings("unchecked") + BatchTransformTranslator typedTranslator = (BatchTransformTranslator) translator; + + // create the applied PTransform on the batchContext + batchContext.setCurrentTransform(AppliedPTransform.of( + node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); + typedTranslator.translateNode(typedTransform, batchContext); + } + + /** + * A translator of a {@link PTransform}. + */ + public interface BatchTransformTranslator { + void translateNode(Type transform, FlinkBatchTranslationContext context); + } + + private static String genSpaces(int n) { + String s = ""; + for (int i = 0; i < n; i++) { + s += "| "; + } + return s; + } + + private static String formatNodeName(TransformTreeNode node) { + return node.toString().split("@")[1] + node.getTransform(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java index 0e45a217d4d2..6a8409c71ddb 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java @@ -93,502 +93,502 @@ */ public class FlinkBatchTransformTranslators { - // -------------------------------------------------------------------------------------------- - // Transform Translator Registry - // -------------------------------------------------------------------------------------------- - - @SuppressWarnings("rawtypes") - private static final Map, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>(); + // -------------------------------------------------------------------------------------------- + // Transform Translator Registry + // -------------------------------------------------------------------------------------------- + + @SuppressWarnings("rawtypes") + private static final Map, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>(); - // register the known translators - static { - TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch()); + // register the known translators + static { + TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch()); - TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch()); - // we don't need this because we translate the Combine.PerKey directly - //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator()); + TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch()); + // we don't need this because we translate the Combine.PerKey directly + //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator()); - TRANSLATORS.put(Create.Values.class, new CreateTranslatorBatch()); + TRANSLATORS.put(Create.Values.class, new CreateTranslatorBatch()); - TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch()); + TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch()); - TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslatorBatch()); - // TODO we're currently ignoring windows here but that has to change in the future - TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); - - TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); - TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch()); - - TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslatorBatch()); - - TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslatorBatch()); - TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslatorBatch()); - - TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch()); - TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslatorBatch()); - - TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslatorBatch()); - TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslatorBatch()); - - // Flink-specific - TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslatorBatch()); - - } - - - public static FlinkBatchPipelineTranslator.BatchTransformTranslator getTranslator(PTransform transform) { - return TRANSLATORS.get(transform.getClass()); - } - - private static class ReadSourceTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - - @Override - public void translateNode(Read.Bounded transform, FlinkBatchTranslationContext context) { - String name = transform.getName(); - BoundedSource source = transform.getSource(); - PCollection output = context.getOutput(transform); - Coder coder = output.getCoder(); - - TypeInformation typeInformation = context.getTypeInfo(output); - - DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), - new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name); - - context.setOutputDataSet(output, dataSource); - } - } - - private static class AvroIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslatorBatch.class); - - @Override - public void translateNode(AvroIO.Read.Bound transform, FlinkBatchTranslationContext context) { - String path = transform.getFilepattern(); - String name = transform.getName(); -// Schema schema = transform.getSchema(); - PValue output = context.getOutput(transform); - - TypeInformation typeInformation = context.getTypeInfo(output); - - // This is super hacky, but unfortunately we cannot get the type otherwise - Class extractedAvroType; - try { - Field typeField = transform.getClass().getDeclaredField("type"); - typeField.setAccessible(true); - @SuppressWarnings("unchecked") - Class avroType = (Class) typeField.get(transform); - extractedAvroType = avroType; - } catch (NoSuchFieldException | IllegalAccessException e) { - // we know that the field is there and it is accessible - throw new RuntimeException("Could not access type from AvroIO.Bound", e); - } - - DataSource source = new DataSource<>(context.getExecutionEnvironment(), - new AvroInputFormat<>(new Path(path), extractedAvroType), - typeInformation, name); - - context.setOutputDataSet(output, source); - } - } - - private static class AvroIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslatorBatch.class); - - @Override - public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); - String filenamePrefix = transform.getFilenamePrefix(); - String filenameSuffix = transform.getFilenameSuffix(); - int numShards = transform.getNumShards(); - String shardNameTemplate = transform.getShardNameTemplate(); - - // TODO: Implement these. We need Flink support for this. - LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", - filenameSuffix); - LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); - - // This is super hacky, but unfortunately we cannot get the type otherwise - Class extractedAvroType; - try { - Field typeField = transform.getClass().getDeclaredField("type"); - typeField.setAccessible(true); - @SuppressWarnings("unchecked") - Class avroType = (Class) typeField.get(transform); - extractedAvroType = avroType; - } catch (NoSuchFieldException | IllegalAccessException e) { - // we know that the field is there and it is accessible - throw new RuntimeException("Could not access type from AvroIO.Bound", e); - } - - DataSink dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path - (filenamePrefix), extractedAvroType)); - - if (numShards > 0) { - dataSink.setParallelism(numShards); - } - } - } - - private static class TextIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslatorBatch.class); - - @Override - public void translateNode(TextIO.Read.Bound transform, FlinkBatchTranslationContext context) { - String path = transform.getFilepattern(); - String name = transform.getName(); - - TextIO.CompressionType compressionType = transform.getCompressionType(); - boolean needsValidation = transform.needsValidation(); - - // TODO: Implement these. We need Flink support for this. - LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType); - LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation); - - PValue output = context.getOutput(transform); - - TypeInformation typeInformation = context.getTypeInfo(output); - DataSource source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name); - - context.setOutputDataSet(output, source); - } - } - - private static class TextIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class); - - @Override - public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslationContext context) { - PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); - - String filenamePrefix = transform.getFilenamePrefix(); - String filenameSuffix = transform.getFilenameSuffix(); - boolean needsValidation = transform.needsValidation(); - int numShards = transform.getNumShards(); - String shardNameTemplate = transform.getShardNameTemplate(); - - // TODO: Implement these. We need Flink support for this. - LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation); - LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix); - LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); - - //inputDataSet.print(); - DataSink dataSink = inputDataSet.writeAsText(filenamePrefix); - - if (numShards > 0) { - dataSink.setParallelism(numShards); - } - } - } - - private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator { - @Override - public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) { - PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); - inputDataSet.printOnTaskManager(transform.getName()); - } - } + TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslatorBatch()); + // TODO we're currently ignoring windows here but that has to change in the future + TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); + + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); + TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch()); + + TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslatorBatch()); + + TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslatorBatch()); + TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslatorBatch()); + + TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch()); + TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslatorBatch()); + + TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslatorBatch()); + TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslatorBatch()); + + // Flink-specific + TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslatorBatch()); + + } + + + public static FlinkBatchPipelineTranslator.BatchTransformTranslator getTranslator(PTransform transform) { + return TRANSLATORS.get(transform.getClass()); + } + + private static class ReadSourceTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + + @Override + public void translateNode(Read.Bounded transform, FlinkBatchTranslationContext context) { + String name = transform.getName(); + BoundedSource source = transform.getSource(); + PCollection output = context.getOutput(transform); + Coder coder = output.getCoder(); + + TypeInformation typeInformation = context.getTypeInfo(output); + + DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), + new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name); + + context.setOutputDataSet(output, dataSource); + } + } + + private static class AvroIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslatorBatch.class); + + @Override + public void translateNode(AvroIO.Read.Bound transform, FlinkBatchTranslationContext context) { + String path = transform.getFilepattern(); + String name = transform.getName(); +// Schema schema = transform.getSchema(); + PValue output = context.getOutput(transform); + + TypeInformation typeInformation = context.getTypeInfo(output); + + // This is super hacky, but unfortunately we cannot get the type otherwise + Class extractedAvroType; + try { + Field typeField = transform.getClass().getDeclaredField("type"); + typeField.setAccessible(true); + @SuppressWarnings("unchecked") + Class avroType = (Class) typeField.get(transform); + extractedAvroType = avroType; + } catch (NoSuchFieldException | IllegalAccessException e) { + // we know that the field is there and it is accessible + throw new RuntimeException("Could not access type from AvroIO.Bound", e); + } + + DataSource source = new DataSource<>(context.getExecutionEnvironment(), + new AvroInputFormat<>(new Path(path), extractedAvroType), + typeInformation, name); + + context.setOutputDataSet(output, source); + } + } + + private static class AvroIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslatorBatch.class); + + @Override + public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslationContext context) { + DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); + String filenamePrefix = transform.getFilenamePrefix(); + String filenameSuffix = transform.getFilenameSuffix(); + int numShards = transform.getNumShards(); + String shardNameTemplate = transform.getShardNameTemplate(); + + // TODO: Implement these. We need Flink support for this. + LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", + filenameSuffix); + LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); + + // This is super hacky, but unfortunately we cannot get the type otherwise + Class extractedAvroType; + try { + Field typeField = transform.getClass().getDeclaredField("type"); + typeField.setAccessible(true); + @SuppressWarnings("unchecked") + Class avroType = (Class) typeField.get(transform); + extractedAvroType = avroType; + } catch (NoSuchFieldException | IllegalAccessException e) { + // we know that the field is there and it is accessible + throw new RuntimeException("Could not access type from AvroIO.Bound", e); + } + + DataSink dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path + (filenamePrefix), extractedAvroType)); + + if (numShards > 0) { + dataSink.setParallelism(numShards); + } + } + } + + private static class TextIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslatorBatch.class); + + @Override + public void translateNode(TextIO.Read.Bound transform, FlinkBatchTranslationContext context) { + String path = transform.getFilepattern(); + String name = transform.getName(); + + TextIO.CompressionType compressionType = transform.getCompressionType(); + boolean needsValidation = transform.needsValidation(); + + // TODO: Implement these. We need Flink support for this. + LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType); + LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation); + + PValue output = context.getOutput(transform); + + TypeInformation typeInformation = context.getTypeInfo(output); + DataSource source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name); + + context.setOutputDataSet(output, source); + } + } + + private static class TextIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class); + + @Override + public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslationContext context) { + PValue input = context.getInput(transform); + DataSet inputDataSet = context.getInputDataSet(input); + + String filenamePrefix = transform.getFilenamePrefix(); + String filenameSuffix = transform.getFilenameSuffix(); + boolean needsValidation = transform.needsValidation(); + int numShards = transform.getNumShards(); + String shardNameTemplate = transform.getShardNameTemplate(); + + // TODO: Implement these. We need Flink support for this. + LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation); + LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix); + LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); + + //inputDataSet.print(); + DataSink dataSink = inputDataSet.writeAsText(filenamePrefix); + + if (numShards > 0) { + dataSink.setParallelism(numShards); + } + } + } + + private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator { + @Override + public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) { + PValue input = context.getInput(transform); + DataSet inputDataSet = context.getInputDataSet(input); + inputDataSet.printOnTaskManager(transform.getName()); + } + } - private static class WriteSinkTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - - @Override - public void translateNode(Write.Bound transform, FlinkBatchTranslationContext context) { - String name = transform.getName(); - PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); + private static class WriteSinkTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + + @Override + public void translateNode(Write.Bound transform, FlinkBatchTranslationContext context) { + String name = transform.getName(); + PValue input = context.getInput(transform); + DataSet inputDataSet = context.getInputDataSet(input); - inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name); - } - } + inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name); + } + } - private static class GroupByKeyOnlyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class GroupByKeyOnlyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - @Override - public void translateNode(GroupByKey.GroupByKeyOnly transform, FlinkBatchTranslationContext context) { - DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); - GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); + @Override + public void translateNode(GroupByKey.GroupByKeyOnly transform, FlinkBatchTranslationContext context) { + DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); + GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); - TypeInformation>> typeInformation = context.getTypeInfo(context.getOutput(transform)); + TypeInformation>> typeInformation = context.getTypeInfo(context.getOutput(transform)); - Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); + Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); - GroupReduceOperator, KV>> outputDataSet = - new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); - context.setOutputDataSet(context.getOutput(transform), outputDataSet); - } - } + GroupReduceOperator, KV>> outputDataSet = + new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + } + } - /** - * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslatorBatch} - */ - private static class GroupByKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + /** + * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslatorBatch} + */ + private static class GroupByKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - @Override - public void translateNode(GroupByKey transform, FlinkBatchTranslationContext context) { - DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); - GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); + @Override + public void translateNode(GroupByKey transform, FlinkBatchTranslationContext context) { + DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); + GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); - TypeInformation>> typeInformation = context.getTypeInfo(context.getOutput(transform)); + TypeInformation>> typeInformation = context.getTypeInfo(context.getOutput(transform)); - Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); + Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); - GroupReduceOperator, KV>> outputDataSet = - new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); + GroupReduceOperator, KV>> outputDataSet = + new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); - context.setOutputDataSet(context.getOutput(transform), outputDataSet); - } - } + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + } + } - private static class CombinePerKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class CombinePerKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - @Override - public void translateNode(Combine.PerKey transform, FlinkBatchTranslationContext context) { - DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); + @Override + public void translateNode(Combine.PerKey transform, FlinkBatchTranslationContext context) { + DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); - @SuppressWarnings("unchecked") - Combine.KeyedCombineFn keyedCombineFn = (Combine.KeyedCombineFn) transform.getFn(); + @SuppressWarnings("unchecked") + Combine.KeyedCombineFn keyedCombineFn = (Combine.KeyedCombineFn) transform.getFn(); - KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); + KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); - Coder accumulatorCoder = - null; - try { - accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder()); - } catch (CannotProvideCoderException e) { - e.printStackTrace(); - // TODO - } + Coder accumulatorCoder = + null; + try { + accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder()); + } catch (CannotProvideCoderException e) { + e.printStackTrace(); + // TODO + } - TypeInformation> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder); - TypeInformation> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder)); + TypeInformation> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder); + TypeInformation> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder)); - Grouping> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation)); + Grouping> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation)); - FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); + FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); - // Partially GroupReduce the values into the intermediate format VA (combine) - GroupCombineOperator, KV> groupCombine = - new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction, - "GroupCombine: " + transform.getName()); + // Partially GroupReduce the values into the intermediate format VA (combine) + GroupCombineOperator, KV> groupCombine = + new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction, + "GroupCombine: " + transform.getName()); - // Reduce fully to VO - GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); + // Reduce fully to VO + GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); - TypeInformation> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform)); + TypeInformation> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform)); - Grouping> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); + Grouping> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); - // Fully reduce the values and create output format VO - GroupReduceOperator, KV> outputDataSet = - new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName()); + // Fully reduce the values and create output format VO + GroupReduceOperator, KV> outputDataSet = + new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName()); - context.setOutputDataSet(context.getOutput(transform), outputDataSet); - } - } + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + } + } -// private static class CombineGroupedValuesTranslator implements FlinkPipelineTranslator.TransformTranslator> { +// private static class CombineGroupedValuesTranslator implements FlinkPipelineTranslator.TransformTranslator> { // -// @Override -// public void translateNode(Combine.GroupedValues transform, TranslationContext context) { -// DataSet> inputDataSet = context.getInputDataSet(transform.getInput()); +// @Override +// public void translateNode(Combine.GroupedValues transform, TranslationContext context) { +// DataSet> inputDataSet = context.getInputDataSet(transform.getInput()); // -// Combine.KeyedCombineFn keyedCombineFn = transform.getFn(); +// Combine.KeyedCombineFn keyedCombineFn = transform.getFn(); // -// GroupReduceFunction, KV> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn); +// GroupReduceFunction, KV> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn); // -// TypeInformation> typeInformation = context.getTypeInfo(transform.getOutput()); +// TypeInformation> typeInformation = context.getTypeInfo(transform.getOutput()); // -// Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType())); +// Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType())); // -// GroupReduceOperator, KV> outputDataSet = -// new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); -// context.setOutputDataSet(transform.getOutput(), outputDataSet); -// } -// } - - private static class ParDoBoundTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorBatch.class); - - @Override - public void translateNode(ParDo.Bound transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); - - final DoFn doFn = transform.getFn(); - - TypeInformation typeInformation = context.getTypeInfo(context.getOutput(transform)); - - FlinkDoFnFunction doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions()); - MapPartitionOperator outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName()); - - transformSideInputs(transform.getSideInputs(), outputDataSet, context); - - context.setOutputDataSet(context.getOutput(transform), outputDataSet); - } - } - - private static class ParDoBoundMultiTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslatorBatch.class); - - @Override - public void translateNode(ParDo.BoundMulti transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); - - final DoFn doFn = transform.getFn(); - - Map, PCollection> outputs = context.getOutput(transform).getAll(); - - Map, Integer> outputMap = Maps.newHashMap(); - // put the main output at index 0, FlinkMultiOutputDoFnFunction also expects this - outputMap.put(transform.getMainOutputTag(), 0); - int count = 1; - for (TupleTag tag: outputs.keySet()) { - if (!outputMap.containsKey(tag)) { - outputMap.put(tag, count++); - } - } - - // collect all output Coders and create a UnionCoder for our tagged outputs - List> outputCoders = Lists.newArrayList(); - for (PCollection coll: outputs.values()) { - outputCoders.add(coll.getCoder()); - } - - UnionCoder unionCoder = UnionCoder.of(outputCoders); - - @SuppressWarnings("unchecked") - TypeInformation typeInformation = new CoderTypeInformation<>(unionCoder); - - @SuppressWarnings("unchecked") - FlinkMultiOutputDoFnFunction doFnWrapper = new FlinkMultiOutputDoFnFunction(doFn, context.getPipelineOptions(), outputMap); - MapPartitionOperator outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName()); - - transformSideInputs(transform.getSideInputs(), outputDataSet, context); - - for (Map.Entry, PCollection> output: outputs.entrySet()) { - TypeInformation outputType = context.getTypeInfo(output.getValue()); - int outputTag = outputMap.get(output.getKey()); - FlinkMultiOutputPruningFunction pruningFunction = new FlinkMultiOutputPruningFunction<>(outputTag); - FlatMapOperator pruningOperator = new - FlatMapOperator<>(outputDataSet, outputType, - pruningFunction, output.getValue().getName()); - context.setOutputDataSet(output.getValue(), pruningOperator); - - } - } - } - - private static class FlattenPCollectionTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - - @Override - public void translateNode(Flatten.FlattenPCollectionList transform, FlinkBatchTranslationContext context) { - List> allInputs = context.getInput(transform).getAll(); - DataSet result = null; - for(PCollection collection : allInputs) { - DataSet current = context.getInputDataSet(collection); - if (result == null) { - result = current; - } else { - result = result.union(current); - } - } - context.setOutputDataSet(context.getOutput(transform), result); - } - } - - private static class CreatePCollectionViewTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - @Override - public void translateNode(View.CreatePCollectionView transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); - PCollectionView input = transform.apply(null); - context.setSideInputDataSet(input, inputDataSet); - } - } - - private static class CreateTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - - @Override - public void translateNode(Create.Values transform, FlinkBatchTranslationContext context) { - TypeInformation typeInformation = context.getOutputTypeInfo(); - Iterable elements = transform.getElements(); - - // we need to serialize the elements to byte arrays, since they might contain - // elements that are not serializable by Java serialization. We deserialize them - // in the FlatMap function using the Coder. - - List serializedElements = Lists.newArrayList(); - Coder coder = context.getOutput(transform).getCoder(); - for (OUT element: elements) { - ByteArrayOutputStream bao = new ByteArrayOutputStream(); - try { - coder.encode(element, bao, Coder.Context.OUTER); - serializedElements.add(bao.toByteArray()); - } catch (IOException e) { - throw new RuntimeException("Could not serialize Create elements using Coder: " + e); - } - } - - DataSet initDataSet = context.getExecutionEnvironment().fromElements(1); - FlinkCreateFunction flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder); - FlatMapOperator outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName()); - - context.setOutputDataSet(context.getOutput(transform), outputDataSet); - } - } - - private static void transformSideInputs(List> sideInputs, - MapPartitionOperator outputDataSet, - FlinkBatchTranslationContext context) { - // get corresponding Flink broadcast DataSets - for(PCollectionView input : sideInputs) { - DataSet broadcastSet = context.getSideInputDataSet(input); - outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId()); - } - } +// GroupReduceOperator, KV> outputDataSet = +// new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); +// context.setOutputDataSet(transform.getOutput(), outputDataSet); +// } +// } + + private static class ParDoBoundTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorBatch.class); + + @Override + public void translateNode(ParDo.Bound transform, FlinkBatchTranslationContext context) { + DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); + + final DoFn doFn = transform.getFn(); + + TypeInformation typeInformation = context.getTypeInfo(context.getOutput(transform)); + + FlinkDoFnFunction doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions()); + MapPartitionOperator outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName()); + + transformSideInputs(transform.getSideInputs(), outputDataSet, context); + + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + } + } + + private static class ParDoBoundMultiTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslatorBatch.class); + + @Override + public void translateNode(ParDo.BoundMulti transform, FlinkBatchTranslationContext context) { + DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); + + final DoFn doFn = transform.getFn(); + + Map, PCollection> outputs = context.getOutput(transform).getAll(); + + Map, Integer> outputMap = Maps.newHashMap(); + // put the main output at index 0, FlinkMultiOutputDoFnFunction also expects this + outputMap.put(transform.getMainOutputTag(), 0); + int count = 1; + for (TupleTag tag: outputs.keySet()) { + if (!outputMap.containsKey(tag)) { + outputMap.put(tag, count++); + } + } + + // collect all output Coders and create a UnionCoder for our tagged outputs + List> outputCoders = Lists.newArrayList(); + for (PCollection coll: outputs.values()) { + outputCoders.add(coll.getCoder()); + } + + UnionCoder unionCoder = UnionCoder.of(outputCoders); + + @SuppressWarnings("unchecked") + TypeInformation typeInformation = new CoderTypeInformation<>(unionCoder); + + @SuppressWarnings("unchecked") + FlinkMultiOutputDoFnFunction doFnWrapper = new FlinkMultiOutputDoFnFunction(doFn, context.getPipelineOptions(), outputMap); + MapPartitionOperator outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName()); + + transformSideInputs(transform.getSideInputs(), outputDataSet, context); + + for (Map.Entry, PCollection> output: outputs.entrySet()) { + TypeInformation outputType = context.getTypeInfo(output.getValue()); + int outputTag = outputMap.get(output.getKey()); + FlinkMultiOutputPruningFunction pruningFunction = new FlinkMultiOutputPruningFunction<>(outputTag); + FlatMapOperator pruningOperator = new + FlatMapOperator<>(outputDataSet, outputType, + pruningFunction, output.getValue().getName()); + context.setOutputDataSet(output.getValue(), pruningOperator); + + } + } + } + + private static class FlattenPCollectionTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + + @Override + public void translateNode(Flatten.FlattenPCollectionList transform, FlinkBatchTranslationContext context) { + List> allInputs = context.getInput(transform).getAll(); + DataSet result = null; + for(PCollection collection : allInputs) { + DataSet current = context.getInputDataSet(collection); + if (result == null) { + result = current; + } else { + result = result.union(current); + } + } + context.setOutputDataSet(context.getOutput(transform), result); + } + } + + private static class CreatePCollectionViewTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + @Override + public void translateNode(View.CreatePCollectionView transform, FlinkBatchTranslationContext context) { + DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); + PCollectionView input = transform.apply(null); + context.setSideInputDataSet(input, inputDataSet); + } + } + + private static class CreateTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + + @Override + public void translateNode(Create.Values transform, FlinkBatchTranslationContext context) { + TypeInformation typeInformation = context.getOutputTypeInfo(); + Iterable elements = transform.getElements(); + + // we need to serialize the elements to byte arrays, since they might contain + // elements that are not serializable by Java serialization. We deserialize them + // in the FlatMap function using the Coder. + + List serializedElements = Lists.newArrayList(); + Coder coder = context.getOutput(transform).getCoder(); + for (OUT element: elements) { + ByteArrayOutputStream bao = new ByteArrayOutputStream(); + try { + coder.encode(element, bao, Coder.Context.OUTER); + serializedElements.add(bao.toByteArray()); + } catch (IOException e) { + throw new RuntimeException("Could not serialize Create elements using Coder: " + e); + } + } + + DataSet initDataSet = context.getExecutionEnvironment().fromElements(1); + FlinkCreateFunction flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder); + FlatMapOperator outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName()); + + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + } + } + + private static void transformSideInputs(List> sideInputs, + MapPartitionOperator outputDataSet, + FlinkBatchTranslationContext context) { + // get corresponding Flink broadcast DataSets + for(PCollectionView input : sideInputs) { + DataSet broadcastSet = context.getSideInputDataSet(input); + outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId()); + } + } // Disabled because it depends on a pending pull request to the DataFlowSDK - /** - * Special composite transform translator. Only called if the CoGroup is two dimensional. - * @param - */ - private static class CoGroupByKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + /** + * Special composite transform translator. Only called if the CoGroup is two dimensional. + * @param + */ + private static class CoGroupByKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - @Override - public void translateNode(CoGroupByKey transform, FlinkBatchTranslationContext context) { - KeyedPCollectionTuple input = context.getInput(transform); + @Override + public void translateNode(CoGroupByKey transform, FlinkBatchTranslationContext context) { + KeyedPCollectionTuple input = context.getInput(transform); - CoGbkResultSchema schema = input.getCoGbkResultSchema(); - List> keyedCollections = input.getKeyedCollections(); + CoGbkResultSchema schema = input.getCoGbkResultSchema(); + List> keyedCollections = input.getKeyedCollections(); - KeyedPCollectionTuple.TaggedKeyedPCollection taggedCollection1 = keyedCollections.get(0); - KeyedPCollectionTuple.TaggedKeyedPCollection taggedCollection2 = keyedCollections.get(1); + KeyedPCollectionTuple.TaggedKeyedPCollection taggedCollection1 = keyedCollections.get(0); + KeyedPCollectionTuple.TaggedKeyedPCollection taggedCollection2 = keyedCollections.get(1); - TupleTag tupleTag1 = taggedCollection1.getTupleTag(); - TupleTag tupleTag2 = taggedCollection2.getTupleTag(); + TupleTag tupleTag1 = taggedCollection1.getTupleTag(); + TupleTag tupleTag2 = taggedCollection2.getTupleTag(); - PCollection> collection1 = taggedCollection1.getCollection(); - PCollection> collection2 = taggedCollection2.getCollection(); + PCollection> collection1 = taggedCollection1.getCollection(); + PCollection> collection2 = taggedCollection2.getCollection(); - DataSet> inputDataSet1 = context.getInputDataSet(collection1); - DataSet> inputDataSet2 = context.getInputDataSet(collection2); + DataSet> inputDataSet1 = context.getInputDataSet(collection1); + DataSet> inputDataSet2 = context.getInputDataSet(collection2); - TypeInformation> typeInfo = context.getOutputTypeInfo(); + TypeInformation> typeInfo = context.getOutputTypeInfo(); - FlinkCoGroupKeyedListAggregator aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2); + FlinkCoGroupKeyedListAggregator aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2); - Keys.ExpressionKeys> keySelector1 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType()); - Keys.ExpressionKeys> keySelector2 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType()); + Keys.ExpressionKeys> keySelector1 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType()); + Keys.ExpressionKeys> keySelector2 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType()); - DataSet> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2, - keySelector1, keySelector2, - aggregator, typeInfo, null, transform.getName()); - context.setOutputDataSet(context.getOutput(transform), out); - } - } + DataSet> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2, + keySelector1, keySelector2, + aggregator, typeInfo, null, transform.getName()); + context.setOutputDataSet(context.getOutput(transform), out); + } + } - // -------------------------------------------------------------------------------------------- - // Miscellaneous - // -------------------------------------------------------------------------------------------- - - private FlinkBatchTransformTranslators() {} + // -------------------------------------------------------------------------------------------- + // Miscellaneous + // -------------------------------------------------------------------------------------------- + + private FlinkBatchTransformTranslators() {} } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java index 1072fa32c551..ec1d6c841f81 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java @@ -38,92 +38,92 @@ import java.util.Map; public class FlinkBatchTranslationContext { - - private final Map> dataSets; - private final Map, DataSet> broadcastDataSets; + + private final Map> dataSets; + private final Map, DataSet> broadcastDataSets; - private final ExecutionEnvironment env; - private final PipelineOptions options; + private final ExecutionEnvironment env; + private final PipelineOptions options; - private AppliedPTransform currentTransform; - - // ------------------------------------------------------------------------ - - public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) { - this.env = env; - this.options = options; - this.dataSets = new HashMap<>(); - this.broadcastDataSets = new HashMap<>(); - } - - // ------------------------------------------------------------------------ - - public ExecutionEnvironment getExecutionEnvironment() { - return env; - } + private AppliedPTransform currentTransform; + + // ------------------------------------------------------------------------ + + public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) { + this.env = env; + this.options = options; + this.dataSets = new HashMap<>(); + this.broadcastDataSets = new HashMap<>(); + } + + // ------------------------------------------------------------------------ + + public ExecutionEnvironment getExecutionEnvironment() { + return env; + } - public PipelineOptions getPipelineOptions() { - return options; - } - - @SuppressWarnings("unchecked") - public DataSet getInputDataSet(PValue value) { - return (DataSet) dataSets.get(value); - } + public PipelineOptions getPipelineOptions() { + return options; + } + + @SuppressWarnings("unchecked") + public DataSet getInputDataSet(PValue value) { + return (DataSet) dataSets.get(value); + } - public void setOutputDataSet(PValue value, DataSet set) { - if (!dataSets.containsKey(value)) { - dataSets.put(value, set); - } - } + public void setOutputDataSet(PValue value, DataSet set) { + if (!dataSets.containsKey(value)) { + dataSets.put(value, set); + } + } - /** - * Sets the AppliedPTransform which carries input/output. - * @param currentTransform - */ - public void setCurrentTransform(AppliedPTransform currentTransform) { - this.currentTransform = currentTransform; - } + /** + * Sets the AppliedPTransform which carries input/output. + * @param currentTransform + */ + public void setCurrentTransform(AppliedPTransform currentTransform) { + this.currentTransform = currentTransform; + } - @SuppressWarnings("unchecked") - public DataSet getSideInputDataSet(PCollectionView value) { - return (DataSet) broadcastDataSets.get(value); - } + @SuppressWarnings("unchecked") + public DataSet getSideInputDataSet(PCollectionView value) { + return (DataSet) broadcastDataSets.get(value); + } - public void setSideInputDataSet(PCollectionView value, DataSet set) { - if (!broadcastDataSets.containsKey(value)) { - broadcastDataSets.put(value, set); - } - } - - @SuppressWarnings("unchecked") - public TypeInformation getTypeInfo(PInput output) { - if (output instanceof TypedPValue) { - Coder outputCoder = ((TypedPValue) output).getCoder(); - if (outputCoder instanceof KvCoder) { - return new KvCoderTypeInformation((KvCoder) outputCoder); - } else { - return new CoderTypeInformation(outputCoder); - } - } - return new GenericTypeInfo<>((Class)Object.class); - } + public void setSideInputDataSet(PCollectionView value, DataSet set) { + if (!broadcastDataSets.containsKey(value)) { + broadcastDataSets.put(value, set); + } + } + + @SuppressWarnings("unchecked") + public TypeInformation getTypeInfo(PInput output) { + if (output instanceof TypedPValue) { + Coder outputCoder = ((TypedPValue) output).getCoder(); + if (outputCoder instanceof KvCoder) { + return new KvCoderTypeInformation((KvCoder) outputCoder); + } else { + return new CoderTypeInformation(outputCoder); + } + } + return new GenericTypeInfo<>((Class)Object.class); + } - public TypeInformation getInputTypeInfo() { - return getTypeInfo(currentTransform.getInput()); - } + public TypeInformation getInputTypeInfo() { + return getTypeInfo(currentTransform.getInput()); + } - public TypeInformation getOutputTypeInfo() { - return getTypeInfo((PValue) currentTransform.getOutput()); - } + public TypeInformation getOutputTypeInfo() { + return getTypeInfo((PValue) currentTransform.getOutput()); + } - @SuppressWarnings("unchecked") - I getInput(PTransform transform) { - return (I) currentTransform.getInput(); - } + @SuppressWarnings("unchecked") + I getInput(PTransform transform) { + return (I) currentTransform.getInput(); + } - @SuppressWarnings("unchecked") - O getOutput(PTransform transform) { - return (O) currentTransform.getOutput(); - } + @SuppressWarnings("unchecked") + O getOutput(PTransform transform) { + return (O) currentTransform.getOutput(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java index b56fe0707326..a6a333bab9a6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java @@ -28,7 +28,7 @@ */ public abstract class FlinkPipelineTranslator implements Pipeline.PipelineVisitor { - public void translate(Pipeline pipeline) { - pipeline.traverseTopologically(this); - } + public void translate(Pipeline pipeline) { + pipeline.traverseTopologically(this); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java index ea9ed140cff5..897303d5e0ee 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java @@ -31,113 +31,113 @@ * */ public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator { - /** The necessary context in the case of a straming job. */ - private final FlinkStreamingTranslationContext streamingContext; - - private int depth = 0; - - /** Composite transform that we want to translate before proceeding with other transforms. */ - private PTransform currentCompositeTransform; - - public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, PipelineOptions options) { - this.streamingContext = new FlinkStreamingTranslationContext(env, options); - } - - // -------------------------------------------------------------------------------------------- - // Pipeline Visitor Methods - // -------------------------------------------------------------------------------------------- - - @Override - public void enterCompositeTransform(TransformTreeNode node) { - System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); - - PTransform transform = node.getTransform(); - if (transform != null && currentCompositeTransform == null) { - - StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); - if (translator != null) { - currentCompositeTransform = transform; - } - } - this.depth++; - } - - @Override - public void leaveCompositeTransform(TransformTreeNode node) { - PTransform transform = node.getTransform(); - if (transform != null && currentCompositeTransform == transform) { - - StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); - if (translator != null) { - System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); - applyStreamingTransform(transform, node, translator); - currentCompositeTransform = null; - } else { - throw new IllegalStateException("Attempted to translate composite transform " + - "but no translator was found: " + currentCompositeTransform); - } - } - this.depth--; - System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node)); - } - - @Override - public void visitTransform(TransformTreeNode node) { - System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node)); - if (currentCompositeTransform != null) { - // ignore it - return; - } - - // get the transformation corresponding to hte node we are - // currently visiting and translate it into its Flink alternative. - - PTransform transform = node.getTransform(); - StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); - if (translator == null) { - System.out.println(node.getTransform().getClass()); - throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); - } - applyStreamingTransform(transform, node, translator); - } - - @Override - public void visitValue(PValue value, TransformTreeNode producer) { - // do nothing here - } - - private > void applyStreamingTransform(PTransform transform, TransformTreeNode node, StreamTransformTranslator translator) { - - @SuppressWarnings("unchecked") - T typedTransform = (T) transform; - - @SuppressWarnings("unchecked") - StreamTransformTranslator typedTranslator = (StreamTransformTranslator) translator; - - // create the applied PTransform on the streamingContext - streamingContext.setCurrentTransform(AppliedPTransform.of( - node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); - typedTranslator.translateNode(typedTransform, streamingContext); - } - - /** - * The interface that every Flink translator of a Beam operator should implement. - * This interface is for streaming jobs. For examples of such translators see - * {@link FlinkStreamingTransformTranslators}. - */ - public interface StreamTransformTranslator { - void translateNode(Type transform, FlinkStreamingTranslationContext context); - } - - private static String genSpaces(int n) { - String s = ""; - for (int i = 0; i < n; i++) { - s += "| "; - } - return s; - } - - private static String formatNodeName(TransformTreeNode node) { - return node.toString().split("@")[1] + node.getTransform(); - } + /** The necessary context in the case of a straming job. */ + private final FlinkStreamingTranslationContext streamingContext; + + private int depth = 0; + + /** Composite transform that we want to translate before proceeding with other transforms. */ + private PTransform currentCompositeTransform; + + public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, PipelineOptions options) { + this.streamingContext = new FlinkStreamingTranslationContext(env, options); + } + + // -------------------------------------------------------------------------------------------- + // Pipeline Visitor Methods + // -------------------------------------------------------------------------------------------- + + @Override + public void enterCompositeTransform(TransformTreeNode node) { + System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node)); + + PTransform transform = node.getTransform(); + if (transform != null && currentCompositeTransform == null) { + + StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); + if (translator != null) { + currentCompositeTransform = transform; + } + } + this.depth++; + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + PTransform transform = node.getTransform(); + if (transform != null && currentCompositeTransform == transform) { + + StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); + if (translator != null) { + System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); + applyStreamingTransform(transform, node, translator); + currentCompositeTransform = null; + } else { + throw new IllegalStateException("Attempted to translate composite transform " + + "but no translator was found: " + currentCompositeTransform); + } + } + this.depth--; + System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node)); + } + + @Override + public void visitTransform(TransformTreeNode node) { + System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node)); + if (currentCompositeTransform != null) { + // ignore it + return; + } + + // get the transformation corresponding to hte node we are + // currently visiting and translate it into its Flink alternative. + + PTransform transform = node.getTransform(); + StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); + if (translator == null) { + System.out.println(node.getTransform().getClass()); + throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); + } + applyStreamingTransform(transform, node, translator); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + // do nothing here + } + + private > void applyStreamingTransform(PTransform transform, TransformTreeNode node, StreamTransformTranslator translator) { + + @SuppressWarnings("unchecked") + T typedTransform = (T) transform; + + @SuppressWarnings("unchecked") + StreamTransformTranslator typedTranslator = (StreamTransformTranslator) translator; + + // create the applied PTransform on the streamingContext + streamingContext.setCurrentTransform(AppliedPTransform.of( + node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform)); + typedTranslator.translateNode(typedTransform, streamingContext); + } + + /** + * The interface that every Flink translator of a Beam operator should implement. + * This interface is for streaming jobs. For examples of such translators see + * {@link FlinkStreamingTransformTranslators}. + */ + public interface StreamTransformTranslator { + void translateNode(Type transform, FlinkStreamingTranslationContext context); + } + + private static String genSpaces(int n) { + String s = ""; + for (int i = 0; i < n; i++) { + s += "| "; + } + return s; + } + + private static String formatNodeName(TransformTreeNode node) { + return node.toString().split("@")[1] + node.getTransform(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java index 99dbedbbeaf3..9fd33be1a7ce 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java @@ -59,346 +59,346 @@ */ public class FlinkStreamingTransformTranslators { - // -------------------------------------------------------------------------------------------- - // Transform Translator Registry - // -------------------------------------------------------------------------------------------- - - @SuppressWarnings("rawtypes") - private static final Map, FlinkStreamingPipelineTranslator.StreamTransformTranslator> TRANSLATORS = new HashMap<>(); - - // here you can find all the available translators. - static { - TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator()); - TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator()); - TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator()); - TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator()); - TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslator()); - TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator()); - TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator()); - TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); - TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator()); - } - - public static FlinkStreamingPipelineTranslator.StreamTransformTranslator getTranslator(PTransform transform) { - return TRANSLATORS.get(transform.getClass()); - } - - // -------------------------------------------------------------------------------------------- - // Transformation Implementations - // -------------------------------------------------------------------------------------------- - - private static class CreateStreamingTranslator implements - FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - - @Override - public void translateNode(Create.Values transform, FlinkStreamingTranslationContext context) { - PCollection output = context.getOutput(transform); - Iterable elements = transform.getElements(); - - // we need to serialize the elements to byte arrays, since they might contain - // elements that are not serializable by Java serialization. We deserialize them - // in the FlatMap function using the Coder. - - List serializedElements = Lists.newArrayList(); - Coder elementCoder = context.getOutput(transform).getCoder(); - for (OUT element: elements) { - ByteArrayOutputStream bao = new ByteArrayOutputStream(); - try { - elementCoder.encode(element, bao, Coder.Context.OUTER); - serializedElements.add(bao.toByteArray()); - } catch (IOException e) { - throw new RuntimeException("Could not serialize Create elements using Coder: " + e); - } - } - - - DataStream initDataSet = context.getExecutionEnvironment().fromElements(1); - - FlinkStreamingCreateFunction createFunction = - new FlinkStreamingCreateFunction<>(serializedElements, elementCoder); - - WindowedValue.ValueOnlyWindowedValueCoder windowCoder = WindowedValue.getValueOnlyCoder(elementCoder); - TypeInformation> outputType = new CoderTypeInformation<>(windowCoder); - - DataStream> outputDataStream = initDataSet.flatMap(createFunction) - .returns(outputType); - - context.setOutputDataStream(context.getOutput(transform), outputDataStream); - } - } - - - private static class TextIOWriteBoundStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class); - - @Override - public void translateNode(TextIO.Write.Bound transform, FlinkStreamingTranslationContext context) { - PValue input = context.getInput(transform); - DataStream> inputDataStream = context.getInputDataStream(input); - - String filenamePrefix = transform.getFilenamePrefix(); - String filenameSuffix = transform.getFilenameSuffix(); - boolean needsValidation = transform.needsValidation(); - int numShards = transform.getNumShards(); - String shardNameTemplate = transform.getShardNameTemplate(); - - // TODO: Implement these. We need Flink support for this. - LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation); - LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix); - LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); - - DataStream dataSink = inputDataStream.flatMap(new FlatMapFunction, String>() { - @Override - public void flatMap(WindowedValue value, Collector out) throws Exception { - out.collect(value.getValue().toString()); - } - }); - DataStreamSink output = dataSink.writeAsText(filenamePrefix, FileSystem.WriteMode.OVERWRITE); - - if (numShards > 0) { - output.setParallelism(numShards); - } - } - } - - private static class UnboundedReadSourceTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - - @Override - public void translateNode(Read.Unbounded transform, FlinkStreamingTranslationContext context) { - PCollection output = context.getOutput(transform); - - DataStream> source; - if (transform.getSource().getClass().equals(UnboundedFlinkSource.class)) { - UnboundedFlinkSource flinkSource = (UnboundedFlinkSource) transform.getSource(); - source = context.getExecutionEnvironment() - .addSource(flinkSource.getFlinkSource()) - .flatMap(new FlatMapFunction>() { - @Override - public void flatMap(String s, Collector> collector) throws Exception { - collector.collect(WindowedValue.of(s, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - } - }); - } else { - source = context.getExecutionEnvironment() - .addSource(new UnboundedSourceWrapper<>(context.getPipelineOptions(), transform)); - } - context.setOutputDataStream(output, source); - } - } - - private static class ParDoBoundStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - - @Override - public void translateNode(ParDo.Bound transform, FlinkStreamingTranslationContext context) { - PCollection output = context.getOutput(transform); - - final WindowingStrategy windowingStrategy = - (WindowingStrategy) - context.getOutput(transform).getWindowingStrategy(); - - WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(), - windowingStrategy.getWindowFn().windowCoder()); - CoderTypeInformation> outputWindowedValueCoder = - new CoderTypeInformation<>(outputStreamCoder); - - FlinkParDoBoundWrapper doFnWrapper = new FlinkParDoBoundWrapper<>( - context.getPipelineOptions(), windowingStrategy, transform.getFn()); - DataStream> inputDataStream = context.getInputDataStream(context.getInput(transform)); - SingleOutputStreamOperator> outDataStream = inputDataStream.flatMap(doFnWrapper) - .returns(outputWindowedValueCoder); - - context.setOutputDataStream(context.getOutput(transform), outDataStream); - } - } - - public static class WindowBoundTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - - @Override - public void translateNode(Window.Bound transform, FlinkStreamingTranslationContext context) { - PValue input = context.getInput(transform); - DataStream> inputDataStream = context.getInputDataStream(input); - - final WindowingStrategy windowingStrategy = - (WindowingStrategy) - context.getOutput(transform).getWindowingStrategy(); - - final WindowFn windowFn = windowingStrategy.getWindowFn(); - - WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder( - context.getInput(transform).getCoder(), windowingStrategy.getWindowFn().windowCoder()); - CoderTypeInformation> outputWindowedValueCoder = - new CoderTypeInformation<>(outputStreamCoder); - - final FlinkParDoBoundWrapper windowDoFnAssigner = new FlinkParDoBoundWrapper<>( - context.getPipelineOptions(), windowingStrategy, createWindowAssigner(windowFn)); - - SingleOutputStreamOperator> windowedStream = - inputDataStream.flatMap(windowDoFnAssigner).returns(outputWindowedValueCoder); - context.setOutputDataStream(context.getOutput(transform), windowedStream); - } - - private static DoFn createWindowAssigner(final WindowFn windowFn) { - return new DoFn() { - - @Override - public void processElement(final ProcessContext c) throws Exception { - Collection windows = windowFn.assignWindows( - windowFn.new AssignContext() { - @Override - public T element() { - return c.element(); - } - - @Override - public Instant timestamp() { - return c.timestamp(); - } - - @Override - public Collection windows() { - return c.windowingInternals().windows(); - } - }); - - c.windowingInternals().outputWindowedValue( - c.element(), c.timestamp(), windows, c.pane()); - } - }; - } - } - - public static class GroupByKeyTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - - @Override - public void translateNode(GroupByKey transform, FlinkStreamingTranslationContext context) { - PValue input = context.getInput(transform); - - DataStream>> inputDataStream = context.getInputDataStream(input); - KvCoder inputKvCoder = (KvCoder) context.getInput(transform).getCoder(); - - KeyedStream>, K> groupByKStream = FlinkGroupByKeyWrapper - .groupStreamByKey(inputDataStream, inputKvCoder); - - DataStream>>> groupedByKNWstream = - FlinkGroupAlsoByWindowWrapper.createForIterable(context.getPipelineOptions(), - context.getInput(transform), groupByKStream); - - context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream); - } - } - - public static class CombinePerKeyTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - - @Override - public void translateNode(Combine.PerKey transform, FlinkStreamingTranslationContext context) { - PValue input = context.getInput(transform); - - DataStream>> inputDataStream = context.getInputDataStream(input); - KvCoder inputKvCoder = (KvCoder) context.getInput(transform).getCoder(); - KvCoder outputKvCoder = (KvCoder) context.getOutput(transform).getCoder(); - - KeyedStream>, K> groupByKStream = FlinkGroupByKeyWrapper - .groupStreamByKey(inputDataStream, inputKvCoder); - - Combine.KeyedCombineFn combineFn = (Combine.KeyedCombineFn) transform.getFn(); - DataStream>> groupedByKNWstream = - FlinkGroupAlsoByWindowWrapper.create(context.getPipelineOptions(), - context.getInput(transform), groupByKStream, combineFn, outputKvCoder); - - context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream); - } - } - - public static class FlattenPCollectionTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - - @Override - public void translateNode(Flatten.FlattenPCollectionList transform, FlinkStreamingTranslationContext context) { - List> allInputs = context.getInput(transform).getAll(); - DataStream result = null; - for (PCollection collection : allInputs) { - DataStream current = context.getInputDataStream(collection); - result = (result == null) ? current : result.union(current); - } - context.setOutputDataStream(context.getOutput(transform), result); - } - } - - public static class ParDoBoundMultiStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { - - private final int MAIN_TAG_INDEX = 0; - - @Override - public void translateNode(ParDo.BoundMulti transform, FlinkStreamingTranslationContext context) { - - // we assume that the transformation does not change the windowing strategy. - WindowingStrategy windowingStrategy = context.getInput(transform).getWindowingStrategy(); - - Map, PCollection> outputs = context.getOutput(transform).getAll(); - Map, Integer> tagsToLabels = transformTupleTagsToLabels( - transform.getMainOutputTag(), outputs.keySet()); - - UnionCoder intermUnionCoder = getIntermUnionCoder(outputs.values()); - WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder( - intermUnionCoder, windowingStrategy.getWindowFn().windowCoder()); - - CoderTypeInformation> intermWindowedValueCoder = - new CoderTypeInformation<>(outputStreamCoder); - - FlinkParDoBoundMultiWrapper doFnWrapper = new FlinkParDoBoundMultiWrapper<>( - context.getPipelineOptions(), windowingStrategy, transform.getFn(), - transform.getMainOutputTag(), tagsToLabels); - - DataStream> inputDataStream = context.getInputDataStream(context.getInput(transform)); - SingleOutputStreamOperator> intermDataStream = - inputDataStream.flatMap(doFnWrapper).returns(intermWindowedValueCoder); - - for (Map.Entry, PCollection> output : outputs.entrySet()) { - final int outputTag = tagsToLabels.get(output.getKey()); - - WindowedValue.WindowedValueCoder coderForTag = WindowedValue.getFullCoder( - output.getValue().getCoder(), - windowingStrategy.getWindowFn().windowCoder()); - - CoderTypeInformation> windowedValueCoder = - new CoderTypeInformation(coderForTag); - - context.setOutputDataStream(output.getValue(), - intermDataStream.filter(new FilterFunction>() { - @Override - public boolean filter(WindowedValue value) throws Exception { - return value.getValue().getUnionTag() == outputTag; - } - }).flatMap(new FlatMapFunction, WindowedValue>() { - @Override - public void flatMap(WindowedValue value, Collector> collector) throws Exception { - collector.collect(WindowedValue.of( - value.getValue().getValue(), - value.getTimestamp(), - value.getWindows(), - value.getPane())); - } - }).returns(windowedValueCoder)); - } - } - - private Map, Integer> transformTupleTagsToLabels(TupleTag mainTag, Set> secondaryTags) { - Map, Integer> tagToLabelMap = Maps.newHashMap(); - tagToLabelMap.put(mainTag, MAIN_TAG_INDEX); - int count = MAIN_TAG_INDEX + 1; - for (TupleTag tag : secondaryTags) { - if (!tagToLabelMap.containsKey(tag)) { - tagToLabelMap.put(tag, count++); - } - } - return tagToLabelMap; - } - - private UnionCoder getIntermUnionCoder(Collection> taggedCollections) { - List> outputCoders = Lists.newArrayList(); - for (PCollection coll : taggedCollections) { - outputCoders.add(coll.getCoder()); - } - return UnionCoder.of(outputCoders); - } - } + // -------------------------------------------------------------------------------------------- + // Transform Translator Registry + // -------------------------------------------------------------------------------------------- + + @SuppressWarnings("rawtypes") + private static final Map, FlinkStreamingPipelineTranslator.StreamTransformTranslator> TRANSLATORS = new HashMap<>(); + + // here you can find all the available translators. + static { + TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator()); + TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator()); + TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator()); + TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator()); + TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslator()); + TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator()); + TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator()); + TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); + TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator()); + } + + public static FlinkStreamingPipelineTranslator.StreamTransformTranslator getTranslator(PTransform transform) { + return TRANSLATORS.get(transform.getClass()); + } + + // -------------------------------------------------------------------------------------------- + // Transformation Implementations + // -------------------------------------------------------------------------------------------- + + private static class CreateStreamingTranslator implements + FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(Create.Values transform, FlinkStreamingTranslationContext context) { + PCollection output = context.getOutput(transform); + Iterable elements = transform.getElements(); + + // we need to serialize the elements to byte arrays, since they might contain + // elements that are not serializable by Java serialization. We deserialize them + // in the FlatMap function using the Coder. + + List serializedElements = Lists.newArrayList(); + Coder elementCoder = context.getOutput(transform).getCoder(); + for (OUT element: elements) { + ByteArrayOutputStream bao = new ByteArrayOutputStream(); + try { + elementCoder.encode(element, bao, Coder.Context.OUTER); + serializedElements.add(bao.toByteArray()); + } catch (IOException e) { + throw new RuntimeException("Could not serialize Create elements using Coder: " + e); + } + } + + + DataStream initDataSet = context.getExecutionEnvironment().fromElements(1); + + FlinkStreamingCreateFunction createFunction = + new FlinkStreamingCreateFunction<>(serializedElements, elementCoder); + + WindowedValue.ValueOnlyWindowedValueCoder windowCoder = WindowedValue.getValueOnlyCoder(elementCoder); + TypeInformation> outputType = new CoderTypeInformation<>(windowCoder); + + DataStream> outputDataStream = initDataSet.flatMap(createFunction) + .returns(outputType); + + context.setOutputDataStream(context.getOutput(transform), outputDataStream); + } + } + + + private static class TextIOWriteBoundStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class); + + @Override + public void translateNode(TextIO.Write.Bound transform, FlinkStreamingTranslationContext context) { + PValue input = context.getInput(transform); + DataStream> inputDataStream = context.getInputDataStream(input); + + String filenamePrefix = transform.getFilenamePrefix(); + String filenameSuffix = transform.getFilenameSuffix(); + boolean needsValidation = transform.needsValidation(); + int numShards = transform.getNumShards(); + String shardNameTemplate = transform.getShardNameTemplate(); + + // TODO: Implement these. We need Flink support for this. + LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation); + LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix); + LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); + + DataStream dataSink = inputDataStream.flatMap(new FlatMapFunction, String>() { + @Override + public void flatMap(WindowedValue value, Collector out) throws Exception { + out.collect(value.getValue().toString()); + } + }); + DataStreamSink output = dataSink.writeAsText(filenamePrefix, FileSystem.WriteMode.OVERWRITE); + + if (numShards > 0) { + output.setParallelism(numShards); + } + } + } + + private static class UnboundedReadSourceTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(Read.Unbounded transform, FlinkStreamingTranslationContext context) { + PCollection output = context.getOutput(transform); + + DataStream> source; + if (transform.getSource().getClass().equals(UnboundedFlinkSource.class)) { + UnboundedFlinkSource flinkSource = (UnboundedFlinkSource) transform.getSource(); + source = context.getExecutionEnvironment() + .addSource(flinkSource.getFlinkSource()) + .flatMap(new FlatMapFunction>() { + @Override + public void flatMap(String s, Collector> collector) throws Exception { + collector.collect(WindowedValue.of(s, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + } + }); + } else { + source = context.getExecutionEnvironment() + .addSource(new UnboundedSourceWrapper<>(context.getPipelineOptions(), transform)); + } + context.setOutputDataStream(output, source); + } + } + + private static class ParDoBoundStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(ParDo.Bound transform, FlinkStreamingTranslationContext context) { + PCollection output = context.getOutput(transform); + + final WindowingStrategy windowingStrategy = + (WindowingStrategy) + context.getOutput(transform).getWindowingStrategy(); + + WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(), + windowingStrategy.getWindowFn().windowCoder()); + CoderTypeInformation> outputWindowedValueCoder = + new CoderTypeInformation<>(outputStreamCoder); + + FlinkParDoBoundWrapper doFnWrapper = new FlinkParDoBoundWrapper<>( + context.getPipelineOptions(), windowingStrategy, transform.getFn()); + DataStream> inputDataStream = context.getInputDataStream(context.getInput(transform)); + SingleOutputStreamOperator> outDataStream = inputDataStream.flatMap(doFnWrapper) + .returns(outputWindowedValueCoder); + + context.setOutputDataStream(context.getOutput(transform), outDataStream); + } + } + + public static class WindowBoundTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(Window.Bound transform, FlinkStreamingTranslationContext context) { + PValue input = context.getInput(transform); + DataStream> inputDataStream = context.getInputDataStream(input); + + final WindowingStrategy windowingStrategy = + (WindowingStrategy) + context.getOutput(transform).getWindowingStrategy(); + + final WindowFn windowFn = windowingStrategy.getWindowFn(); + + WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder( + context.getInput(transform).getCoder(), windowingStrategy.getWindowFn().windowCoder()); + CoderTypeInformation> outputWindowedValueCoder = + new CoderTypeInformation<>(outputStreamCoder); + + final FlinkParDoBoundWrapper windowDoFnAssigner = new FlinkParDoBoundWrapper<>( + context.getPipelineOptions(), windowingStrategy, createWindowAssigner(windowFn)); + + SingleOutputStreamOperator> windowedStream = + inputDataStream.flatMap(windowDoFnAssigner).returns(outputWindowedValueCoder); + context.setOutputDataStream(context.getOutput(transform), windowedStream); + } + + private static DoFn createWindowAssigner(final WindowFn windowFn) { + return new DoFn() { + + @Override + public void processElement(final ProcessContext c) throws Exception { + Collection windows = windowFn.assignWindows( + windowFn.new AssignContext() { + @Override + public T element() { + return c.element(); + } + + @Override + public Instant timestamp() { + return c.timestamp(); + } + + @Override + public Collection windows() { + return c.windowingInternals().windows(); + } + }); + + c.windowingInternals().outputWindowedValue( + c.element(), c.timestamp(), windows, c.pane()); + } + }; + } + } + + public static class GroupByKeyTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(GroupByKey transform, FlinkStreamingTranslationContext context) { + PValue input = context.getInput(transform); + + DataStream>> inputDataStream = context.getInputDataStream(input); + KvCoder inputKvCoder = (KvCoder) context.getInput(transform).getCoder(); + + KeyedStream>, K> groupByKStream = FlinkGroupByKeyWrapper + .groupStreamByKey(inputDataStream, inputKvCoder); + + DataStream>>> groupedByKNWstream = + FlinkGroupAlsoByWindowWrapper.createForIterable(context.getPipelineOptions(), + context.getInput(transform), groupByKStream); + + context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream); + } + } + + public static class CombinePerKeyTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(Combine.PerKey transform, FlinkStreamingTranslationContext context) { + PValue input = context.getInput(transform); + + DataStream>> inputDataStream = context.getInputDataStream(input); + KvCoder inputKvCoder = (KvCoder) context.getInput(transform).getCoder(); + KvCoder outputKvCoder = (KvCoder) context.getOutput(transform).getCoder(); + + KeyedStream>, K> groupByKStream = FlinkGroupByKeyWrapper + .groupStreamByKey(inputDataStream, inputKvCoder); + + Combine.KeyedCombineFn combineFn = (Combine.KeyedCombineFn) transform.getFn(); + DataStream>> groupedByKNWstream = + FlinkGroupAlsoByWindowWrapper.create(context.getPipelineOptions(), + context.getInput(transform), groupByKStream, combineFn, outputKvCoder); + + context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream); + } + } + + public static class FlattenPCollectionTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + @Override + public void translateNode(Flatten.FlattenPCollectionList transform, FlinkStreamingTranslationContext context) { + List> allInputs = context.getInput(transform).getAll(); + DataStream result = null; + for (PCollection collection : allInputs) { + DataStream current = context.getInputDataStream(collection); + result = (result == null) ? current : result.union(current); + } + context.setOutputDataStream(context.getOutput(transform), result); + } + } + + public static class ParDoBoundMultiStreamingTranslator implements FlinkStreamingPipelineTranslator.StreamTransformTranslator> { + + private final int MAIN_TAG_INDEX = 0; + + @Override + public void translateNode(ParDo.BoundMulti transform, FlinkStreamingTranslationContext context) { + + // we assume that the transformation does not change the windowing strategy. + WindowingStrategy windowingStrategy = context.getInput(transform).getWindowingStrategy(); + + Map, PCollection> outputs = context.getOutput(transform).getAll(); + Map, Integer> tagsToLabels = transformTupleTagsToLabels( + transform.getMainOutputTag(), outputs.keySet()); + + UnionCoder intermUnionCoder = getIntermUnionCoder(outputs.values()); + WindowedValue.WindowedValueCoder outputStreamCoder = WindowedValue.getFullCoder( + intermUnionCoder, windowingStrategy.getWindowFn().windowCoder()); + + CoderTypeInformation> intermWindowedValueCoder = + new CoderTypeInformation<>(outputStreamCoder); + + FlinkParDoBoundMultiWrapper doFnWrapper = new FlinkParDoBoundMultiWrapper<>( + context.getPipelineOptions(), windowingStrategy, transform.getFn(), + transform.getMainOutputTag(), tagsToLabels); + + DataStream> inputDataStream = context.getInputDataStream(context.getInput(transform)); + SingleOutputStreamOperator> intermDataStream = + inputDataStream.flatMap(doFnWrapper).returns(intermWindowedValueCoder); + + for (Map.Entry, PCollection> output : outputs.entrySet()) { + final int outputTag = tagsToLabels.get(output.getKey()); + + WindowedValue.WindowedValueCoder coderForTag = WindowedValue.getFullCoder( + output.getValue().getCoder(), + windowingStrategy.getWindowFn().windowCoder()); + + CoderTypeInformation> windowedValueCoder = + new CoderTypeInformation(coderForTag); + + context.setOutputDataStream(output.getValue(), + intermDataStream.filter(new FilterFunction>() { + @Override + public boolean filter(WindowedValue value) throws Exception { + return value.getValue().getUnionTag() == outputTag; + } + }).flatMap(new FlatMapFunction, WindowedValue>() { + @Override + public void flatMap(WindowedValue value, Collector> collector) throws Exception { + collector.collect(WindowedValue.of( + value.getValue().getValue(), + value.getTimestamp(), + value.getWindows(), + value.getPane())); + } + }).returns(windowedValueCoder)); + } + } + + private Map, Integer> transformTupleTagsToLabels(TupleTag mainTag, Set> secondaryTags) { + Map, Integer> tagToLabelMap = Maps.newHashMap(); + tagToLabelMap.put(mainTag, MAIN_TAG_INDEX); + int count = MAIN_TAG_INDEX + 1; + for (TupleTag tag : secondaryTags) { + if (!tagToLabelMap.containsKey(tag)) { + tagToLabelMap.put(tag, count++); + } + } + return tagToLabelMap; + } + + private UnionCoder getIntermUnionCoder(Collection> taggedCollections) { + List> outputCoders = Lists.newArrayList(); + for (PCollection coll : taggedCollections) { + outputCoders.add(coll.getCoder()); + } + return UnionCoder.of(outputCoders); + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java index 7c4ab93dbbe5..3586d0c871c7 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java @@ -30,58 +30,58 @@ public class FlinkStreamingTranslationContext { - private final StreamExecutionEnvironment env; - private final PipelineOptions options; + private final StreamExecutionEnvironment env; + private final PipelineOptions options; - /** - * Keeps a mapping between the output value of the PTransform (in Dataflow) and the - * Flink Operator that produced it, after the translation of the correspondinf PTransform - * to its Flink equivalent. - * */ - private final Map> dataStreams; + /** + * Keeps a mapping between the output value of the PTransform (in Dataflow) and the + * Flink Operator that produced it, after the translation of the correspondinf PTransform + * to its Flink equivalent. + * */ + private final Map> dataStreams; - private AppliedPTransform currentTransform; + private AppliedPTransform currentTransform; - public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) { - this.env = Preconditions.checkNotNull(env); - this.options = Preconditions.checkNotNull(options); - this.dataStreams = new HashMap<>(); - } + public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) { + this.env = Preconditions.checkNotNull(env); + this.options = Preconditions.checkNotNull(options); + this.dataStreams = new HashMap<>(); + } - public StreamExecutionEnvironment getExecutionEnvironment() { - return env; - } + public StreamExecutionEnvironment getExecutionEnvironment() { + return env; + } - public PipelineOptions getPipelineOptions() { - return options; - } + public PipelineOptions getPipelineOptions() { + return options; + } - @SuppressWarnings("unchecked") - public DataStream getInputDataStream(PValue value) { - return (DataStream) dataStreams.get(value); - } + @SuppressWarnings("unchecked") + public DataStream getInputDataStream(PValue value) { + return (DataStream) dataStreams.get(value); + } - public void setOutputDataStream(PValue value, DataStream set) { - if (!dataStreams.containsKey(value)) { - dataStreams.put(value, set); - } - } + public void setOutputDataStream(PValue value, DataStream set) { + if (!dataStreams.containsKey(value)) { + dataStreams.put(value, set); + } + } - /** - * Sets the AppliedPTransform which carries input/output. - * @param currentTransform - */ - public void setCurrentTransform(AppliedPTransform currentTransform) { - this.currentTransform = currentTransform; - } + /** + * Sets the AppliedPTransform which carries input/output. + * @param currentTransform + */ + public void setCurrentTransform(AppliedPTransform currentTransform) { + this.currentTransform = currentTransform; + } - @SuppressWarnings("unchecked") - public I getInput(PTransform transform) { - return (I) currentTransform.getInput(); - } + @SuppressWarnings("unchecked") + public I getInput(PTransform transform) { + return (I) currentTransform.getInput(); + } - @SuppressWarnings("unchecked") - public O getOutput(PTransform transform) { - return (O) currentTransform.getOutput(); - } + @SuppressWarnings("unchecked") + public O getOutput(PTransform transform) { + return (O) currentTransform.getOutput(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java index 4c7fefd6b9b6..5897473fc699 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java @@ -29,30 +29,30 @@ public class FlinkCoGroupKeyedListAggregator implements CoGroupFunction, KV, KV>{ - private CoGbkResultSchema schema; - private TupleTag tupleTag1; - private TupleTag tupleTag2; + private CoGbkResultSchema schema; + private TupleTag tupleTag1; + private TupleTag tupleTag2; - public FlinkCoGroupKeyedListAggregator(CoGbkResultSchema schema, TupleTag tupleTag1, TupleTag tupleTag2) { - this.schema = schema; - this.tupleTag1 = tupleTag1; - this.tupleTag2 = tupleTag2; - } + public FlinkCoGroupKeyedListAggregator(CoGbkResultSchema schema, TupleTag tupleTag1, TupleTag tupleTag2) { + this.schema = schema; + this.tupleTag1 = tupleTag1; + this.tupleTag2 = tupleTag2; + } - @Override - public void coGroup(Iterable> first, Iterable> second, Collector> out) throws Exception { - K k = null; - List result = new ArrayList<>(); - int index1 = schema.getIndex(tupleTag1); - for (KV entry : first) { - k = entry.getKey(); - result.add(new RawUnionValue(index1, entry.getValue())); - } - int index2 = schema.getIndex(tupleTag2); - for (KV entry : second) { - k = entry.getKey(); - result.add(new RawUnionValue(index2, entry.getValue())); - } - out.collect(KV.of(k, new CoGbkResult(schema, result))); - } + @Override + public void coGroup(Iterable> first, Iterable> second, Collector> out) throws Exception { + K k = null; + List result = new ArrayList<>(); + int index1 = schema.getIndex(tupleTag1); + for (KV entry : first) { + k = entry.getKey(); + result.add(new RawUnionValue(index1, entry.getValue())); + } + int index2 = schema.getIndex(tupleTag2); + for (KV entry : second) { + k = entry.getKey(); + result.add(new RawUnionValue(index2, entry.getValue())); + } + out.collect(KV.of(k, new CoGbkResult(schema, result))); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java index 21ecaf0a26bc..03f2b06b4cc3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java @@ -32,29 +32,29 @@ */ public class FlinkCreateFunction implements FlatMapFunction { - private final List elements; - private final Coder coder; - - public FlinkCreateFunction(List elements, Coder coder) { - this.elements = elements; - this.coder = coder; - } - - @Override - @SuppressWarnings("unchecked") - public void flatMap(IN value, Collector out) throws Exception { - - for (byte[] element : elements) { - ByteArrayInputStream bai = new ByteArrayInputStream(element); - OUT outValue = coder.decode(bai, Coder.Context.OUTER); - if (outValue == null) { - // TODO Flink doesn't allow null values in records - out.collect((OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE); - } else { - out.collect(outValue); - } - } - - out.close(); - } + private final List elements; + private final Coder coder; + + public FlinkCreateFunction(List elements, Coder coder) { + this.elements = elements; + this.coder = coder; + } + + @Override + @SuppressWarnings("unchecked") + public void flatMap(IN value, Collector out) throws Exception { + + for (byte[] element : elements) { + ByteArrayInputStream bai = new ByteArrayInputStream(element); + OUT outValue = coder.decode(bai, Coder.Context.OUTER); + if (outValue == null) { + // TODO Flink doesn't allow null values in records + out.collect((OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE); + } else { + out.collect(outValue); + } + } + + out.close(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java index 9c57d4e0ee96..53ff1cf4e5ff 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java @@ -49,154 +49,154 @@ */ public class FlinkDoFnFunction extends RichMapPartitionFunction { - private final DoFn doFn; - private transient PipelineOptions options; - - public FlinkDoFnFunction(DoFn doFn, PipelineOptions options) { - this.doFn = doFn; - this.options = options; - } - - private void writeObject(ObjectOutputStream out) - throws IOException, ClassNotFoundException { - out.defaultWriteObject(); - ObjectMapper mapper = new ObjectMapper(); - mapper.writeValue(out, options); - } - - private void readObject(ObjectInputStream in) - throws IOException, ClassNotFoundException { - in.defaultReadObject(); - ObjectMapper mapper = new ObjectMapper(); - options = mapper.readValue(in, PipelineOptions.class); - } - - @Override - public void mapPartition(Iterable values, Collector out) throws Exception { - ProcessContext context = new ProcessContext(doFn, out); - this.doFn.startBundle(context); - for (IN value : values) { - context.inValue = value; - doFn.processElement(context); - } - this.doFn.finishBundle(context); - } - - private class ProcessContext extends DoFn.ProcessContext { - - IN inValue; - Collector outCollector; - - public ProcessContext(DoFn fn, Collector outCollector) { - fn.super(); - super.setupDelegateAggregators(); - this.outCollector = outCollector; - } - - @Override - public IN element() { - return this.inValue; - } - - - @Override - public Instant timestamp() { - return Instant.now(); - } - - @Override - public BoundedWindow window() { - return GlobalWindow.INSTANCE; - } - - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } - - @Override - public WindowingInternals windowingInternals() { - return new WindowingInternals() { - @Override - public StateInternals stateInternals() { - return null; - } - - @Override - public void outputWindowedValue(OUT output, Instant timestamp, Collection windows, PaneInfo pane) { - - } - - @Override - public TimerInternals timerInternals() { - return null; - } - - @Override - public Collection windows() { - return ImmutableList.of(GlobalWindow.INSTANCE); - } - - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } - - @Override - public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { - } - - @Override - public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { - throw new RuntimeException("sideInput() not implemented."); - } - }; - } - - @Override - public PipelineOptions getPipelineOptions() { - return options; - } - - @Override - public T sideInput(PCollectionView view) { - List sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal().getId()); - List> windowedValueList = new ArrayList<>(sideInput.size()); - for (T input : sideInput) { - windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane())); - } - return view.fromIterableInternal(windowedValueList); - } - - @Override - public void output(OUT output) { - outCollector.collect(output); - } - - @Override - public void outputWithTimestamp(OUT output, Instant timestamp) { - // not FLink's way, just output normally - output(output); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - // ignore the side output, this can happen when a user does not register - // side outputs but then outputs using a freshly created TupleTag. - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - sideOutput(tag, output); - } - - @Override - protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { - SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper<>(combiner); - getRuntimeContext().addAccumulator(name, wrapper); - return wrapper; - } - - - } + private final DoFn doFn; + private transient PipelineOptions options; + + public FlinkDoFnFunction(DoFn doFn, PipelineOptions options) { + this.doFn = doFn; + this.options = options; + } + + private void writeObject(ObjectOutputStream out) + throws IOException, ClassNotFoundException { + out.defaultWriteObject(); + ObjectMapper mapper = new ObjectMapper(); + mapper.writeValue(out, options); + } + + private void readObject(ObjectInputStream in) + throws IOException, ClassNotFoundException { + in.defaultReadObject(); + ObjectMapper mapper = new ObjectMapper(); + options = mapper.readValue(in, PipelineOptions.class); + } + + @Override + public void mapPartition(Iterable values, Collector out) throws Exception { + ProcessContext context = new ProcessContext(doFn, out); + this.doFn.startBundle(context); + for (IN value : values) { + context.inValue = value; + doFn.processElement(context); + } + this.doFn.finishBundle(context); + } + + private class ProcessContext extends DoFn.ProcessContext { + + IN inValue; + Collector outCollector; + + public ProcessContext(DoFn fn, Collector outCollector) { + fn.super(); + super.setupDelegateAggregators(); + this.outCollector = outCollector; + } + + @Override + public IN element() { + return this.inValue; + } + + + @Override + public Instant timestamp() { + return Instant.now(); + } + + @Override + public BoundedWindow window() { + return GlobalWindow.INSTANCE; + } + + @Override + public PaneInfo pane() { + return PaneInfo.NO_FIRING; + } + + @Override + public WindowingInternals windowingInternals() { + return new WindowingInternals() { + @Override + public StateInternals stateInternals() { + return null; + } + + @Override + public void outputWindowedValue(OUT output, Instant timestamp, Collection windows, PaneInfo pane) { + + } + + @Override + public TimerInternals timerInternals() { + return null; + } + + @Override + public Collection windows() { + return ImmutableList.of(GlobalWindow.INSTANCE); + } + + @Override + public PaneInfo pane() { + return PaneInfo.NO_FIRING; + } + + @Override + public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { + } + + @Override + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + throw new RuntimeException("sideInput() not implemented."); + } + }; + } + + @Override + public PipelineOptions getPipelineOptions() { + return options; + } + + @Override + public T sideInput(PCollectionView view) { + List sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal().getId()); + List> windowedValueList = new ArrayList<>(sideInput.size()); + for (T input : sideInput) { + windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane())); + } + return view.fromIterableInternal(windowedValueList); + } + + @Override + public void output(OUT output) { + outCollector.collect(output); + } + + @Override + public void outputWithTimestamp(OUT output, Instant timestamp) { + // not FLink's way, just output normally + output(output); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + // ignore the side output, this can happen when a user does not register + // side outputs but then outputs using a freshly created TupleTag. + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + sideOutput(tag, output); + } + + @Override + protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { + SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper<>(combiner); + getRuntimeContext().addAccumulator(name, wrapper); + return wrapper; + } + + + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java index 5d3702a483c6..011697293eef 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java @@ -29,47 +29,47 @@ */ public class FlinkKeyedListAggregationFunction implements GroupReduceFunction, KV>> { - @Override - public void reduce(Iterable> values, Collector>> out) throws Exception { - Iterator> it = values.iterator(); - KV first = it.next(); - Iterable passThrough = new PassThroughIterable<>(first, it); - out.collect(KV.of(first.getKey(), passThrough)); - } + @Override + public void reduce(Iterable> values, Collector>> out) throws Exception { + Iterator> it = values.iterator(); + KV first = it.next(); + Iterable passThrough = new PassThroughIterable<>(first, it); + out.collect(KV.of(first.getKey(), passThrough)); + } - private static class PassThroughIterable implements Iterable, Iterator { - private KV first; - private Iterator> iterator; + private static class PassThroughIterable implements Iterable, Iterator { + private KV first; + private Iterator> iterator; - public PassThroughIterable(KV first, Iterator> iterator) { - this.first = first; - this.iterator = iterator; - } + public PassThroughIterable(KV first, Iterator> iterator) { + this.first = first; + this.iterator = iterator; + } - @Override - public Iterator iterator() { - return this; - } + @Override + public Iterator iterator() { + return this; + } - @Override - public boolean hasNext() { - return first != null || iterator.hasNext(); - } + @Override + public boolean hasNext() { + return first != null || iterator.hasNext(); + } - @Override - public V next() { - if (first != null) { - V result = first.getValue(); - first = null; - return result; - } else { - return iterator.next().getValue(); - } - } + @Override + public V next() { + if (first != null) { + V result = first.getValue(); + first = null; + return result; + } else { + return iterator.next().getValue(); + } + } - @Override - public void remove() { - throw new UnsupportedOperationException("Cannot remove elements from input."); - } - } + @Override + public void remove() { + throw new UnsupportedOperationException("Cannot remove elements from input."); + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java index 6187182c34cc..9e516389e95d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -51,125 +51,125 @@ */ public class FlinkMultiOutputDoFnFunction extends RichMapPartitionFunction { - private final DoFn doFn; - private transient PipelineOptions options; - private final Map, Integer> outputMap; - - public FlinkMultiOutputDoFnFunction(DoFn doFn, PipelineOptions options, Map, Integer> outputMap) { - this.doFn = doFn; - this.options = options; - this.outputMap = outputMap; - } - - private void writeObject(ObjectOutputStream out) - throws IOException, ClassNotFoundException { - out.defaultWriteObject(); - ObjectMapper mapper = new ObjectMapper(); - mapper.writeValue(out, options); - } - - private void readObject(ObjectInputStream in) - throws IOException, ClassNotFoundException { - in.defaultReadObject(); - ObjectMapper mapper = new ObjectMapper(); - options = mapper.readValue(in, PipelineOptions.class); - - } - - @Override - public void mapPartition(Iterable values, Collector out) throws Exception { - ProcessContext context = new ProcessContext(doFn, out); - this.doFn.startBundle(context); - for (IN value : values) { - context.inValue = value; - doFn.processElement(context); - } - this.doFn.finishBundle(context); - } - - private class ProcessContext extends DoFn.ProcessContext { - - IN inValue; - Collector outCollector; - - public ProcessContext(DoFn fn, Collector outCollector) { - fn.super(); - this.outCollector = outCollector; - } - - @Override - public IN element() { - return this.inValue; - } - - @Override - public Instant timestamp() { - return Instant.now(); - } - - @Override - public BoundedWindow window() { - return GlobalWindow.INSTANCE; - } - - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } - - @Override - public WindowingInternals windowingInternals() { - return null; - } - - @Override - public PipelineOptions getPipelineOptions() { - return options; - } - - @Override - public T sideInput(PCollectionView view) { - List sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal() - .getId()); - List> windowedValueList = new ArrayList<>(sideInput.size()); - for (T input : sideInput) { - windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane())); - } - return view.fromIterableInternal(windowedValueList); - } - - @Override - public void output(OUT value) { - // assume that index 0 is the default output - outCollector.collect(new RawUnionValue(0, value)); - } - - @Override - public void outputWithTimestamp(OUT output, Instant timestamp) { - // not FLink's way, just output normally - output(output); - } - - @Override - @SuppressWarnings("unchecked") - public void sideOutput(TupleTag tag, T value) { - Integer index = outputMap.get(tag); - if (index != null) { - outCollector.collect(new RawUnionValue(index, value)); - } - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - sideOutput(tag, output); - } - - @Override - protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { - SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper<>(combiner); - getRuntimeContext().addAccumulator(name, wrapper); - return null; - } - - } + private final DoFn doFn; + private transient PipelineOptions options; + private final Map, Integer> outputMap; + + public FlinkMultiOutputDoFnFunction(DoFn doFn, PipelineOptions options, Map, Integer> outputMap) { + this.doFn = doFn; + this.options = options; + this.outputMap = outputMap; + } + + private void writeObject(ObjectOutputStream out) + throws IOException, ClassNotFoundException { + out.defaultWriteObject(); + ObjectMapper mapper = new ObjectMapper(); + mapper.writeValue(out, options); + } + + private void readObject(ObjectInputStream in) + throws IOException, ClassNotFoundException { + in.defaultReadObject(); + ObjectMapper mapper = new ObjectMapper(); + options = mapper.readValue(in, PipelineOptions.class); + + } + + @Override + public void mapPartition(Iterable values, Collector out) throws Exception { + ProcessContext context = new ProcessContext(doFn, out); + this.doFn.startBundle(context); + for (IN value : values) { + context.inValue = value; + doFn.processElement(context); + } + this.doFn.finishBundle(context); + } + + private class ProcessContext extends DoFn.ProcessContext { + + IN inValue; + Collector outCollector; + + public ProcessContext(DoFn fn, Collector outCollector) { + fn.super(); + this.outCollector = outCollector; + } + + @Override + public IN element() { + return this.inValue; + } + + @Override + public Instant timestamp() { + return Instant.now(); + } + + @Override + public BoundedWindow window() { + return GlobalWindow.INSTANCE; + } + + @Override + public PaneInfo pane() { + return PaneInfo.NO_FIRING; + } + + @Override + public WindowingInternals windowingInternals() { + return null; + } + + @Override + public PipelineOptions getPipelineOptions() { + return options; + } + + @Override + public T sideInput(PCollectionView view) { + List sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal() + .getId()); + List> windowedValueList = new ArrayList<>(sideInput.size()); + for (T input : sideInput) { + windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane())); + } + return view.fromIterableInternal(windowedValueList); + } + + @Override + public void output(OUT value) { + // assume that index 0 is the default output + outCollector.collect(new RawUnionValue(0, value)); + } + + @Override + public void outputWithTimestamp(OUT output, Instant timestamp) { + // not FLink's way, just output normally + output(output); + } + + @Override + @SuppressWarnings("unchecked") + public void sideOutput(TupleTag tag, T value) { + Integer index = outputMap.get(tag); + if (index != null) { + outCollector.collect(new RawUnionValue(index, value)); + } + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + sideOutput(tag, output); + } + + @Override + protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { + SerializableFnAggregatorWrapper wrapper = new SerializableFnAggregatorWrapper<>(combiner); + getRuntimeContext().addAccumulator(name, wrapper); + return null; + } + + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java index 6792b23ec93b..e883d42bc97b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java @@ -25,17 +25,17 @@ */ public class FlinkMultiOutputPruningFunction implements FlatMapFunction { - private final int outputTag; + private final int outputTag; - public FlinkMultiOutputPruningFunction(int outputTag) { - this.outputTag = outputTag; - } + public FlinkMultiOutputPruningFunction(int outputTag) { + this.outputTag = outputTag; + } - @Override - @SuppressWarnings("unchecked") - public void flatMap(RawUnionValue rawUnionValue, Collector collector) throws Exception { - if (rawUnionValue.getUnionTag() == outputTag) { - collector.collect((T) rawUnionValue.getValue()); - } - } + @Override + @SuppressWarnings("unchecked") + public void flatMap(RawUnionValue rawUnionValue, Collector collector) throws Exception { + if (rawUnionValue.getUnionTag() == outputTag) { + collector.collect((T) rawUnionValue.getValue()); + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java index ef47b7257a80..1ff9aff88ab6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java @@ -32,29 +32,29 @@ */ public class FlinkPartialReduceFunction implements GroupCombineFunction, KV> { - private final Combine.KeyedCombineFn keyedCombineFn; - - public FlinkPartialReduceFunction(Combine.KeyedCombineFn - keyedCombineFn) { - this.keyedCombineFn = keyedCombineFn; - } - - @Override - public void combine(Iterable> elements, Collector> out) throws Exception { - - final Iterator> iterator = elements.iterator(); - // create accumulator using the first elements key - KV first = iterator.next(); - K key = first.getKey(); - VI value = first.getValue(); - VA accumulator = keyedCombineFn.createAccumulator(key); - accumulator = keyedCombineFn.addInput(key, accumulator, value); - - while(iterator.hasNext()) { - value = iterator.next().getValue(); - accumulator = keyedCombineFn.addInput(key, accumulator, value); - } - - out.collect(KV.of(key, accumulator)); - } + private final Combine.KeyedCombineFn keyedCombineFn; + + public FlinkPartialReduceFunction(Combine.KeyedCombineFn + keyedCombineFn) { + this.keyedCombineFn = keyedCombineFn; + } + + @Override + public void combine(Iterable> elements, Collector> out) throws Exception { + + final Iterator> iterator = elements.iterator(); + // create accumulator using the first elements key + KV first = iterator.next(); + K key = first.getKey(); + VI value = first.getValue(); + VA accumulator = keyedCombineFn.createAccumulator(key); + accumulator = keyedCombineFn.addInput(key, accumulator, value); + + while(iterator.hasNext()) { + value = iterator.next().getValue(); + accumulator = keyedCombineFn.addInput(key, accumulator, value); + } + + out.collect(KV.of(key, accumulator)); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java index cd0b38c183bb..94676a2896dc 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java @@ -33,25 +33,25 @@ */ public class FlinkReduceFunction implements GroupReduceFunction, KV> { - private final Combine.KeyedCombineFn keyedCombineFn; + private final Combine.KeyedCombineFn keyedCombineFn; - public FlinkReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { - this.keyedCombineFn = keyedCombineFn; - } + public FlinkReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { + this.keyedCombineFn = keyedCombineFn; + } - @Override - public void reduce(Iterable> values, Collector> out) throws Exception { - Iterator> it = values.iterator(); + @Override + public void reduce(Iterable> values, Collector> out) throws Exception { + Iterator> it = values.iterator(); - KV current = it.next(); - K k = current.getKey(); - VA accumulator = current.getValue(); + KV current = it.next(); + K k = current.getKey(); + VA accumulator = current.getValue(); - while (it.hasNext()) { - current = it.next(); - keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) ); - } + while (it.hasNext()) { + current = it.next(); + keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) ); + } - out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator))); - } + out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator))); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java index 0befa8841ff6..3cc5c2440570 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java @@ -38,113 +38,113 @@ */ @SuppressWarnings("serial") public class UnionCoder extends StandardCoder { - // TODO: Think about how to integrate this with a schema object (i.e. - // a tuple of tuple tags). - /** - * Builds a union coder with the given list of element coders. This list - * corresponds to a mapping of union tag to Coder. Union tags start at 0. - */ - public static UnionCoder of(List> elementCoders) { - return new UnionCoder(elementCoders); - } - - @JsonCreator - public static UnionCoder jsonOf( - @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) - List> elements) { - return UnionCoder.of(elements); - } - - private int getIndexForEncoding(RawUnionValue union) { - if (union == null) { - throw new IllegalArgumentException("cannot encode a null tagged union"); - } - int index = union.getUnionTag(); - if (index < 0 || index >= elementCoders.size()) { - throw new IllegalArgumentException( - "union value index " + index + " not in range [0.." + - (elementCoders.size() - 1) + "]"); - } - return index; - } - - @SuppressWarnings("unchecked") - @Override - public void encode( - RawUnionValue union, - OutputStream outStream, - Context context) - throws IOException { - int index = getIndexForEncoding(union); - // Write out the union tag. - VarInt.encode(index, outStream); - - // Write out the actual value. - Coder coder = (Coder) elementCoders.get(index); - coder.encode( - union.getValue(), - outStream, - context); - } - - @Override - public RawUnionValue decode(InputStream inStream, Context context) - throws IOException { - int index = VarInt.decodeInt(inStream); - Object value = elementCoders.get(index).decode(inStream, context); - return new RawUnionValue(index, value); - } - - @Override - public List> getCoderArguments() { - return null; - } - - @Override - public List> getComponents() { - return elementCoders; - } - - /** - * Since this coder uses elementCoders.get(index) and coders that are known to run in constant - * time, we defer the return value to that coder. - */ - @Override - public boolean isRegisterByteSizeObserverCheap(RawUnionValue union, Context context) { - int index = getIndexForEncoding(union); - @SuppressWarnings("unchecked") - Coder coder = (Coder) elementCoders.get(index); - return coder.isRegisterByteSizeObserverCheap(union.getValue(), context); - } - - /** - * Notifies ElementByteSizeObserver about the byte size of the encoded value using this coder. - */ - @Override - public void registerByteSizeObserver( - RawUnionValue union, ElementByteSizeObserver observer, Context context) - throws Exception { - int index = getIndexForEncoding(union); - // Write out the union tag. - observer.update(VarInt.getLength(index)); - // Write out the actual value. - @SuppressWarnings("unchecked") - Coder coder = (Coder) elementCoders.get(index); - coder.registerByteSizeObserver(union.getValue(), observer, context); - } - - ///////////////////////////////////////////////////////////////////////////// - - private final List> elementCoders; - - private UnionCoder(List> elementCoders) { - this.elementCoders = elementCoders; - } - - @Override - public void verifyDeterministic() throws NonDeterministicException { - verifyDeterministic( - "UnionCoder is only deterministic if all element coders are", - elementCoders); - } + // TODO: Think about how to integrate this with a schema object (i.e. + // a tuple of tuple tags). + /** + * Builds a union coder with the given list of element coders. This list + * corresponds to a mapping of union tag to Coder. Union tags start at 0. + */ + public static UnionCoder of(List> elementCoders) { + return new UnionCoder(elementCoders); + } + + @JsonCreator + public static UnionCoder jsonOf( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> elements) { + return UnionCoder.of(elements); + } + + private int getIndexForEncoding(RawUnionValue union) { + if (union == null) { + throw new IllegalArgumentException("cannot encode a null tagged union"); + } + int index = union.getUnionTag(); + if (index < 0 || index >= elementCoders.size()) { + throw new IllegalArgumentException( + "union value index " + index + " not in range [0.." + + (elementCoders.size() - 1) + "]"); + } + return index; + } + + @SuppressWarnings("unchecked") + @Override + public void encode( + RawUnionValue union, + OutputStream outStream, + Context context) + throws IOException { + int index = getIndexForEncoding(union); + // Write out the union tag. + VarInt.encode(index, outStream); + + // Write out the actual value. + Coder coder = (Coder) elementCoders.get(index); + coder.encode( + union.getValue(), + outStream, + context); + } + + @Override + public RawUnionValue decode(InputStream inStream, Context context) + throws IOException { + int index = VarInt.decodeInt(inStream); + Object value = elementCoders.get(index).decode(inStream, context); + return new RawUnionValue(index, value); + } + + @Override + public List> getCoderArguments() { + return null; + } + + @Override + public List> getComponents() { + return elementCoders; + } + + /** + * Since this coder uses elementCoders.get(index) and coders that are known to run in constant + * time, we defer the return value to that coder. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(RawUnionValue union, Context context) { + int index = getIndexForEncoding(union); + @SuppressWarnings("unchecked") + Coder coder = (Coder) elementCoders.get(index); + return coder.isRegisterByteSizeObserverCheap(union.getValue(), context); + } + + /** + * Notifies ElementByteSizeObserver about the byte size of the encoded value using this coder. + */ + @Override + public void registerByteSizeObserver( + RawUnionValue union, ElementByteSizeObserver observer, Context context) + throws Exception { + int index = getIndexForEncoding(union); + // Write out the union tag. + observer.update(VarInt.getLength(index)); + // Write out the actual value. + @SuppressWarnings("unchecked") + Coder coder = (Coder) elementCoders.get(index); + coder.registerByteSizeObserver(union.getValue(), observer, context); + } + + ///////////////////////////////////////////////////////////////////////////// + + private final List> elementCoders; + + private UnionCoder(List> elementCoders) { + this.elementCoders = elementCoders; + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic( + "UnionCoder is only deterministic if all element coders are", + elementCoders); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java index e43358944564..b402f7c4f30d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java @@ -32,185 +32,185 @@ */ public class CoderComparator extends TypeComparator { - private Coder coder; - - // We use these for internal encoding/decoding for creating copies and comparing - // serialized forms using a Coder - private transient InspectableByteArrayOutputStream buffer1; - private transient InspectableByteArrayOutputStream buffer2; - - // For storing the Reference in encoded form - private transient InspectableByteArrayOutputStream referenceBuffer; - - public CoderComparator(Coder coder) { - this.coder = coder; - buffer1 = new InspectableByteArrayOutputStream(); - buffer2 = new InspectableByteArrayOutputStream(); - referenceBuffer = new InspectableByteArrayOutputStream(); - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - buffer1 = new InspectableByteArrayOutputStream(); - buffer2 = new InspectableByteArrayOutputStream(); - referenceBuffer = new InspectableByteArrayOutputStream(); - } - - @Override - public int hash(T record) { - return record.hashCode(); - } - - @Override - public void setReference(T toCompare) { - referenceBuffer.reset(); - try { - coder.encode(toCompare, referenceBuffer, Coder.Context.OUTER); - } catch (IOException e) { - throw new RuntimeException("Could not set reference " + toCompare + ": " + e); - } - } - - @Override - public boolean equalToReference(T candidate) { - try { - buffer2.reset(); - coder.encode(candidate, buffer2, Coder.Context.OUTER); - byte[] arr = referenceBuffer.getBuffer(); - byte[] arrOther = buffer2.getBuffer(); - if (referenceBuffer.size() != buffer2.size()) { - return false; - } - int len = buffer2.size(); - for(int i = 0; i < len; i++ ) { - if (arr[i] != arrOther[i]) { - return false; - } - } - return true; - } catch (IOException e) { - throw new RuntimeException("Could not compare reference.", e); - } - } - - @Override - public int compareToReference(TypeComparator other) { - InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComparator) other).referenceBuffer; - - byte[] arr = referenceBuffer.getBuffer(); - byte[] arrOther = otherReferenceBuffer.getBuffer(); - if (referenceBuffer.size() != otherReferenceBuffer.size()) { - return referenceBuffer.size() - otherReferenceBuffer.size(); - } - int len = referenceBuffer.size(); - for (int i = 0; i < len; i++) { - if (arr[i] != arrOther[i]) { - return arr[i] - arrOther[i]; - } - } - return 0; - } - - @Override - public int compare(T first, T second) { - try { - buffer1.reset(); - buffer2.reset(); - coder.encode(first, buffer1, Coder.Context.OUTER); - coder.encode(second, buffer2, Coder.Context.OUTER); - byte[] arr = buffer1.getBuffer(); - byte[] arrOther = buffer2.getBuffer(); - if (buffer1.size() != buffer2.size()) { - return buffer1.size() - buffer2.size(); - } - int len = buffer1.size(); - for(int i = 0; i < len; i++ ) { - if (arr[i] != arrOther[i]) { - return arr[i] - arrOther[i]; - } - } - return 0; - } catch (IOException e) { - throw new RuntimeException("Could not compare: ", e); - } - } - - @Override - public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { - CoderTypeSerializer serializer = new CoderTypeSerializer<>(coder); - T first = serializer.deserialize(firstSource); - T second = serializer.deserialize(secondSource); - return compare(first, second); - } - - @Override - public boolean supportsNormalizedKey() { - return true; - } - - @Override - public boolean supportsSerializationWithKeyNormalization() { - return false; - } - - @Override - public int getNormalizeKeyLen() { - return Integer.MAX_VALUE; - } - - @Override - public boolean isNormalizedKeyPrefixOnly(int keyBytes) { - return true; - } - - @Override - public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) { - buffer1.reset(); - try { - coder.encode(record, buffer1, Coder.Context.OUTER); - } catch (IOException e) { - throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e); - } - final byte[] data = buffer1.getBuffer(); - final int limit = offset + numBytes; - - target.put(offset, data, 0, Math.min(numBytes, buffer1.size())); - - offset += buffer1.size(); - - while (offset < limit) { - target.put(offset++, (byte) 0); - } - } - - @Override - public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public boolean invertNormalizedKey() { - return false; - } - - @Override - public TypeComparator duplicate() { - return new CoderComparator<>(coder); - } - - @Override - public int extractKeys(Object record, Object[] target, int index) { - target[index] = record; - return 1; - } - - @Override - public TypeComparator[] getFlatComparators() { - return new TypeComparator[] { this.duplicate() }; - } + private Coder coder; + + // We use these for internal encoding/decoding for creating copies and comparing + // serialized forms using a Coder + private transient InspectableByteArrayOutputStream buffer1; + private transient InspectableByteArrayOutputStream buffer2; + + // For storing the Reference in encoded form + private transient InspectableByteArrayOutputStream referenceBuffer; + + public CoderComparator(Coder coder) { + this.coder = coder; + buffer1 = new InspectableByteArrayOutputStream(); + buffer2 = new InspectableByteArrayOutputStream(); + referenceBuffer = new InspectableByteArrayOutputStream(); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + buffer1 = new InspectableByteArrayOutputStream(); + buffer2 = new InspectableByteArrayOutputStream(); + referenceBuffer = new InspectableByteArrayOutputStream(); + } + + @Override + public int hash(T record) { + return record.hashCode(); + } + + @Override + public void setReference(T toCompare) { + referenceBuffer.reset(); + try { + coder.encode(toCompare, referenceBuffer, Coder.Context.OUTER); + } catch (IOException e) { + throw new RuntimeException("Could not set reference " + toCompare + ": " + e); + } + } + + @Override + public boolean equalToReference(T candidate) { + try { + buffer2.reset(); + coder.encode(candidate, buffer2, Coder.Context.OUTER); + byte[] arr = referenceBuffer.getBuffer(); + byte[] arrOther = buffer2.getBuffer(); + if (referenceBuffer.size() != buffer2.size()) { + return false; + } + int len = buffer2.size(); + for(int i = 0; i < len; i++ ) { + if (arr[i] != arrOther[i]) { + return false; + } + } + return true; + } catch (IOException e) { + throw new RuntimeException("Could not compare reference.", e); + } + } + + @Override + public int compareToReference(TypeComparator other) { + InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComparator) other).referenceBuffer; + + byte[] arr = referenceBuffer.getBuffer(); + byte[] arrOther = otherReferenceBuffer.getBuffer(); + if (referenceBuffer.size() != otherReferenceBuffer.size()) { + return referenceBuffer.size() - otherReferenceBuffer.size(); + } + int len = referenceBuffer.size(); + for (int i = 0; i < len; i++) { + if (arr[i] != arrOther[i]) { + return arr[i] - arrOther[i]; + } + } + return 0; + } + + @Override + public int compare(T first, T second) { + try { + buffer1.reset(); + buffer2.reset(); + coder.encode(first, buffer1, Coder.Context.OUTER); + coder.encode(second, buffer2, Coder.Context.OUTER); + byte[] arr = buffer1.getBuffer(); + byte[] arrOther = buffer2.getBuffer(); + if (buffer1.size() != buffer2.size()) { + return buffer1.size() - buffer2.size(); + } + int len = buffer1.size(); + for(int i = 0; i < len; i++ ) { + if (arr[i] != arrOther[i]) { + return arr[i] - arrOther[i]; + } + } + return 0; + } catch (IOException e) { + throw new RuntimeException("Could not compare: ", e); + } + } + + @Override + public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { + CoderTypeSerializer serializer = new CoderTypeSerializer<>(coder); + T first = serializer.deserialize(firstSource); + T second = serializer.deserialize(secondSource); + return compare(first, second); + } + + @Override + public boolean supportsNormalizedKey() { + return true; + } + + @Override + public boolean supportsSerializationWithKeyNormalization() { + return false; + } + + @Override + public int getNormalizeKeyLen() { + return Integer.MAX_VALUE; + } + + @Override + public boolean isNormalizedKeyPrefixOnly(int keyBytes) { + return true; + } + + @Override + public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) { + buffer1.reset(); + try { + coder.encode(record, buffer1, Coder.Context.OUTER); + } catch (IOException e) { + throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e); + } + final byte[] data = buffer1.getBuffer(); + final int limit = offset + numBytes; + + target.put(offset, data, 0, Math.min(numBytes, buffer1.size())); + + offset += buffer1.size(); + + while (offset < limit) { + target.put(offset++, (byte) 0); + } + } + + @Override + public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean invertNormalizedKey() { + return false; + } + + @Override + public TypeComparator duplicate() { + return new CoderComparator<>(coder); + } + + @Override + public int extractKeys(Object record, Object[] target, int index) { + target[index] = record; + return 1; + } + + @Override + public TypeComparator[] getFlatComparators() { + return new TypeComparator[] { this.duplicate() }; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java index dd9c5f684585..ae4309ea02dd 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java @@ -32,85 +32,85 @@ */ public class CoderTypeInformation extends TypeInformation implements AtomicType { - private final Coder coder; - - public CoderTypeInformation(Coder coder) { - Preconditions.checkNotNull(coder); - this.coder = coder; - } - - @Override - public boolean isBasicType() { - return false; - } - - @Override - public boolean isTupleType() { - return false; - } - - @Override - public int getArity() { - return 1; - } - - @Override - @SuppressWarnings("unchecked") - public Class getTypeClass() { - // We don't have the Class, so we have to pass null here. What a shame... - return (Class) Object.class; - } - - @Override - public boolean isKeyType() { - return true; - } - - @Override - @SuppressWarnings("unchecked") - public TypeSerializer createSerializer(ExecutionConfig config) { - if (coder instanceof VoidCoder) { - return (TypeSerializer) new VoidCoderTypeSerializer(); - } - return new CoderTypeSerializer<>(coder); - } - - @Override - public int getTotalFields() { - return 2; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - CoderTypeInformation that = (CoderTypeInformation) o; - - return coder.equals(that.coder); - - } - - @Override - public int hashCode() { - return coder.hashCode(); - } - - @Override - public boolean canEqual(Object obj) { - return obj instanceof CoderTypeInformation; - } - - @Override - public String toString() { - return "CoderTypeInformation{" + - "coder=" + coder + - '}'; - } - - @Override - public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig - executionConfig) { - return new CoderComparator<>(coder); - } + private final Coder coder; + + public CoderTypeInformation(Coder coder) { + Preconditions.checkNotNull(coder); + this.coder = coder; + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 1; + } + + @Override + @SuppressWarnings("unchecked") + public Class getTypeClass() { + // We don't have the Class, so we have to pass null here. What a shame... + return (Class) Object.class; + } + + @Override + public boolean isKeyType() { + return true; + } + + @Override + @SuppressWarnings("unchecked") + public TypeSerializer createSerializer(ExecutionConfig config) { + if (coder instanceof VoidCoder) { + return (TypeSerializer) new VoidCoderTypeSerializer(); + } + return new CoderTypeSerializer<>(coder); + } + + @Override + public int getTotalFields() { + return 2; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + CoderTypeInformation that = (CoderTypeInformation) o; + + return coder.equals(that.coder); + + } + + @Override + public int hashCode() { + return coder.hashCode(); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof CoderTypeInformation; + } + + @Override + public String toString() { + return "CoderTypeInformation{" + + "coder=" + coder + + '}'; + } + + @Override + public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig + executionConfig) { + return new CoderComparator<>(coder); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java index f739397a4846..6ed661c33ab0 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java @@ -35,118 +35,118 @@ * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s */ public class CoderTypeSerializer extends TypeSerializer { - - private Coder coder; - private transient DataInputViewWrapper inputWrapper; - private transient DataOutputViewWrapper outputWrapper; - - // We use this for internal encoding/decoding for creating copies using the Coder. - private transient InspectableByteArrayOutputStream buffer; - - public CoderTypeSerializer(Coder coder) { - this.coder = coder; - this.inputWrapper = new DataInputViewWrapper(null); - this.outputWrapper = new DataOutputViewWrapper(null); - - buffer = new InspectableByteArrayOutputStream(); - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - this.inputWrapper = new DataInputViewWrapper(null); - this.outputWrapper = new DataOutputViewWrapper(null); - - buffer = new InspectableByteArrayOutputStream(); - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public CoderTypeSerializer duplicate() { - return new CoderTypeSerializer<>(coder); - } - - @Override - public T createInstance() { - return null; - } - - @Override - public T copy(T t) { - buffer.reset(); - try { - coder.encode(t, buffer, Coder.Context.OUTER); - } catch (IOException e) { - throw new RuntimeException("Could not copy.", e); - } - try { - return coder.decode(new ByteArrayInputStream(buffer.getBuffer(), 0, buffer - .size()), Coder.Context.OUTER); - } catch (IOException e) { - throw new RuntimeException("Could not copy.", e); - } - } - - @Override - public T copy(T t, T reuse) { - return copy(t); - } - - @Override - public int getLength() { - return 0; - } - - @Override - public void serialize(T t, DataOutputView dataOutputView) throws IOException { - outputWrapper.setOutputView(dataOutputView); - coder.encode(t, outputWrapper, Coder.Context.NESTED); - } - - @Override - public T deserialize(DataInputView dataInputView) throws IOException { - try { - inputWrapper.setInputView(dataInputView); - return coder.decode(inputWrapper, Coder.Context.NESTED); - } catch (CoderException e) { - Throwable cause = e.getCause(); - if (cause instanceof EOFException) { - throw (EOFException) cause; - } else { - throw e; - } - } - } - - @Override - public T deserialize(T t, DataInputView dataInputView) throws IOException { - return deserialize(dataInputView); - } - - @Override - public void copy(DataInputView dataInputView, DataOutputView dataOutputView) throws IOException { - serialize(deserialize(dataInputView), dataOutputView); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - CoderTypeSerializer that = (CoderTypeSerializer) o; - return coder.equals(that.coder); - } - - @Override - public boolean canEqual(Object obj) { - return obj instanceof CoderTypeSerializer; - } - - @Override - public int hashCode() { - return coder.hashCode(); - } + + private Coder coder; + private transient DataInputViewWrapper inputWrapper; + private transient DataOutputViewWrapper outputWrapper; + + // We use this for internal encoding/decoding for creating copies using the Coder. + private transient InspectableByteArrayOutputStream buffer; + + public CoderTypeSerializer(Coder coder) { + this.coder = coder; + this.inputWrapper = new DataInputViewWrapper(null); + this.outputWrapper = new DataOutputViewWrapper(null); + + buffer = new InspectableByteArrayOutputStream(); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + this.inputWrapper = new DataInputViewWrapper(null); + this.outputWrapper = new DataOutputViewWrapper(null); + + buffer = new InspectableByteArrayOutputStream(); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public CoderTypeSerializer duplicate() { + return new CoderTypeSerializer<>(coder); + } + + @Override + public T createInstance() { + return null; + } + + @Override + public T copy(T t) { + buffer.reset(); + try { + coder.encode(t, buffer, Coder.Context.OUTER); + } catch (IOException e) { + throw new RuntimeException("Could not copy.", e); + } + try { + return coder.decode(new ByteArrayInputStream(buffer.getBuffer(), 0, buffer + .size()), Coder.Context.OUTER); + } catch (IOException e) { + throw new RuntimeException("Could not copy.", e); + } + } + + @Override + public T copy(T t, T reuse) { + return copy(t); + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(T t, DataOutputView dataOutputView) throws IOException { + outputWrapper.setOutputView(dataOutputView); + coder.encode(t, outputWrapper, Coder.Context.NESTED); + } + + @Override + public T deserialize(DataInputView dataInputView) throws IOException { + try { + inputWrapper.setInputView(dataInputView); + return coder.decode(inputWrapper, Coder.Context.NESTED); + } catch (CoderException e) { + Throwable cause = e.getCause(); + if (cause instanceof EOFException) { + throw (EOFException) cause; + } else { + throw e; + } + } + } + + @Override + public T deserialize(T t, DataInputView dataInputView) throws IOException { + return deserialize(dataInputView); + } + + @Override + public void copy(DataInputView dataInputView, DataOutputView dataOutputView) throws IOException { + serialize(deserialize(dataInputView), dataOutputView); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + CoderTypeSerializer that = (CoderTypeSerializer) o; + return coder.equals(that.coder); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof CoderTypeSerializer; + } + + @Override + public int hashCode() { + return coder.hashCode(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java index 5d918cc7e9d1..be6eadd160ff 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java @@ -25,10 +25,10 @@ */ public class InspectableByteArrayOutputStream extends ByteArrayOutputStream { - /** - * Get the underlying byte array. - */ - public byte[] getBuffer() { - return buf; - } + /** + * Get the underlying byte array. + */ + public byte[] getBuffer() { + return buf; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index 815569d09d85..ba09ea91f361 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -35,230 +35,230 @@ * for {@link KV} that always compares on the key only. */ public class KvCoderComperator extends TypeComparator> { - - private KvCoder coder; - private Coder keyCoder; - - // We use these for internal encoding/decoding for creating copies and comparing - // serialized forms using a Coder - private transient InspectableByteArrayOutputStream buffer1; - private transient InspectableByteArrayOutputStream buffer2; - - // For storing the Reference in encoded form - private transient InspectableByteArrayOutputStream referenceBuffer; - - - // For deserializing the key - private transient DataInputViewWrapper inputWrapper; - - public KvCoderComperator(KvCoder coder) { - this.coder = coder; - this.keyCoder = coder.getKeyCoder(); - - buffer1 = new InspectableByteArrayOutputStream(); - buffer2 = new InspectableByteArrayOutputStream(); - referenceBuffer = new InspectableByteArrayOutputStream(); - - inputWrapper = new DataInputViewWrapper(null); - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - - buffer1 = new InspectableByteArrayOutputStream(); - buffer2 = new InspectableByteArrayOutputStream(); - referenceBuffer = new InspectableByteArrayOutputStream(); - - inputWrapper = new DataInputViewWrapper(null); - } - - @Override - public int hash(KV record) { - K key = record.getKey(); - if (key != null) { - return key.hashCode(); - } else { - return 0; - } - } - - @Override - public void setReference(KV toCompare) { - referenceBuffer.reset(); - try { - keyCoder.encode(toCompare.getKey(), referenceBuffer, Coder.Context.OUTER); - } catch (IOException e) { - throw new RuntimeException("Could not set reference " + toCompare + ": " + e); - } - } - - @Override - public boolean equalToReference(KV candidate) { - try { - buffer2.reset(); - keyCoder.encode(candidate.getKey(), buffer2, Coder.Context.OUTER); - byte[] arr = referenceBuffer.getBuffer(); - byte[] arrOther = buffer2.getBuffer(); - if (referenceBuffer.size() != buffer2.size()) { - return false; - } - int len = buffer2.size(); - for(int i = 0; i < len; i++ ) { - if (arr[i] != arrOther[i]) { - return false; - } - } - return true; - } catch (IOException e) { - throw new RuntimeException("Could not compare reference.", e); - } - } - - @Override - public int compareToReference(TypeComparator> other) { - InspectableByteArrayOutputStream otherReferenceBuffer = ((KvCoderComperator) other).referenceBuffer; - - byte[] arr = referenceBuffer.getBuffer(); - byte[] arrOther = otherReferenceBuffer.getBuffer(); - if (referenceBuffer.size() != otherReferenceBuffer.size()) { - return referenceBuffer.size() - otherReferenceBuffer.size(); - } - int len = referenceBuffer.size(); - for (int i = 0; i < len; i++) { - if (arr[i] != arrOther[i]) { - return arr[i] - arrOther[i]; - } - } - return 0; - } - - - @Override - public int compare(KV first, KV second) { - try { - buffer1.reset(); - buffer2.reset(); - keyCoder.encode(first.getKey(), buffer1, Coder.Context.OUTER); - keyCoder.encode(second.getKey(), buffer2, Coder.Context.OUTER); - byte[] arr = buffer1.getBuffer(); - byte[] arrOther = buffer2.getBuffer(); - if (buffer1.size() != buffer2.size()) { - return buffer1.size() - buffer2.size(); - } - int len = buffer1.size(); - for(int i = 0; i < len; i++ ) { - if (arr[i] != arrOther[i]) { - return arr[i] - arrOther[i]; - } - } - return 0; - } catch (IOException e) { - throw new RuntimeException("Could not compare reference.", e); - } - } - - @Override - public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { - - inputWrapper.setInputView(firstSource); - K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); - inputWrapper.setInputView(secondSource); - K secondKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); - - try { - buffer1.reset(); - buffer2.reset(); - keyCoder.encode(firstKey, buffer1, Coder.Context.OUTER); - keyCoder.encode(secondKey, buffer2, Coder.Context.OUTER); - byte[] arr = buffer1.getBuffer(); - byte[] arrOther = buffer2.getBuffer(); - if (buffer1.size() != buffer2.size()) { - return buffer1.size() - buffer2.size(); - } - int len = buffer1.size(); - for(int i = 0; i < len; i++ ) { - if (arr[i] != arrOther[i]) { - return arr[i] - arrOther[i]; - } - } - return 0; - } catch (IOException e) { - throw new RuntimeException("Could not compare reference.", e); - } - } - - @Override - public boolean supportsNormalizedKey() { - return true; - } - - @Override - public boolean supportsSerializationWithKeyNormalization() { - return false; - } - - @Override - public int getNormalizeKeyLen() { - return Integer.MAX_VALUE; - } - - @Override - public boolean isNormalizedKeyPrefixOnly(int keyBytes) { - return true; - } - - @Override - public void putNormalizedKey(KV record, MemorySegment target, int offset, int numBytes) { - buffer1.reset(); - try { - keyCoder.encode(record.getKey(), buffer1, Coder.Context.NESTED); - } catch (IOException e) { - throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e); - } - final byte[] data = buffer1.getBuffer(); - final int limit = offset + numBytes; - - int numBytesPut = Math.min(numBytes, buffer1.size()); - - target.put(offset, data, 0, numBytesPut); - - offset += numBytesPut; - - while (offset < limit) { - target.put(offset++, (byte) 0); - } - } - - @Override - public void writeWithKeyNormalization(KV record, DataOutputView target) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public KV readWithKeyDenormalization(KV reuse, DataInputView source) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public boolean invertNormalizedKey() { - return false; - } - - @Override - public TypeComparator> duplicate() { - return new KvCoderComperator<>(coder); - } - - @Override - public int extractKeys(Object record, Object[] target, int index) { - KV kv = (KV) record; - K k = kv.getKey(); - target[index] = k; - return 1; - } - - @Override - public TypeComparator[] getFlatComparators() { - return new TypeComparator[] {new CoderComparator<>(keyCoder)}; - } + + private KvCoder coder; + private Coder keyCoder; + + // We use these for internal encoding/decoding for creating copies and comparing + // serialized forms using a Coder + private transient InspectableByteArrayOutputStream buffer1; + private transient InspectableByteArrayOutputStream buffer2; + + // For storing the Reference in encoded form + private transient InspectableByteArrayOutputStream referenceBuffer; + + + // For deserializing the key + private transient DataInputViewWrapper inputWrapper; + + public KvCoderComperator(KvCoder coder) { + this.coder = coder; + this.keyCoder = coder.getKeyCoder(); + + buffer1 = new InspectableByteArrayOutputStream(); + buffer2 = new InspectableByteArrayOutputStream(); + referenceBuffer = new InspectableByteArrayOutputStream(); + + inputWrapper = new DataInputViewWrapper(null); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + + buffer1 = new InspectableByteArrayOutputStream(); + buffer2 = new InspectableByteArrayOutputStream(); + referenceBuffer = new InspectableByteArrayOutputStream(); + + inputWrapper = new DataInputViewWrapper(null); + } + + @Override + public int hash(KV record) { + K key = record.getKey(); + if (key != null) { + return key.hashCode(); + } else { + return 0; + } + } + + @Override + public void setReference(KV toCompare) { + referenceBuffer.reset(); + try { + keyCoder.encode(toCompare.getKey(), referenceBuffer, Coder.Context.OUTER); + } catch (IOException e) { + throw new RuntimeException("Could not set reference " + toCompare + ": " + e); + } + } + + @Override + public boolean equalToReference(KV candidate) { + try { + buffer2.reset(); + keyCoder.encode(candidate.getKey(), buffer2, Coder.Context.OUTER); + byte[] arr = referenceBuffer.getBuffer(); + byte[] arrOther = buffer2.getBuffer(); + if (referenceBuffer.size() != buffer2.size()) { + return false; + } + int len = buffer2.size(); + for(int i = 0; i < len; i++ ) { + if (arr[i] != arrOther[i]) { + return false; + } + } + return true; + } catch (IOException e) { + throw new RuntimeException("Could not compare reference.", e); + } + } + + @Override + public int compareToReference(TypeComparator> other) { + InspectableByteArrayOutputStream otherReferenceBuffer = ((KvCoderComperator) other).referenceBuffer; + + byte[] arr = referenceBuffer.getBuffer(); + byte[] arrOther = otherReferenceBuffer.getBuffer(); + if (referenceBuffer.size() != otherReferenceBuffer.size()) { + return referenceBuffer.size() - otherReferenceBuffer.size(); + } + int len = referenceBuffer.size(); + for (int i = 0; i < len; i++) { + if (arr[i] != arrOther[i]) { + return arr[i] - arrOther[i]; + } + } + return 0; + } + + + @Override + public int compare(KV first, KV second) { + try { + buffer1.reset(); + buffer2.reset(); + keyCoder.encode(first.getKey(), buffer1, Coder.Context.OUTER); + keyCoder.encode(second.getKey(), buffer2, Coder.Context.OUTER); + byte[] arr = buffer1.getBuffer(); + byte[] arrOther = buffer2.getBuffer(); + if (buffer1.size() != buffer2.size()) { + return buffer1.size() - buffer2.size(); + } + int len = buffer1.size(); + for(int i = 0; i < len; i++ ) { + if (arr[i] != arrOther[i]) { + return arr[i] - arrOther[i]; + } + } + return 0; + } catch (IOException e) { + throw new RuntimeException("Could not compare reference.", e); + } + } + + @Override + public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { + + inputWrapper.setInputView(firstSource); + K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); + inputWrapper.setInputView(secondSource); + K secondKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); + + try { + buffer1.reset(); + buffer2.reset(); + keyCoder.encode(firstKey, buffer1, Coder.Context.OUTER); + keyCoder.encode(secondKey, buffer2, Coder.Context.OUTER); + byte[] arr = buffer1.getBuffer(); + byte[] arrOther = buffer2.getBuffer(); + if (buffer1.size() != buffer2.size()) { + return buffer1.size() - buffer2.size(); + } + int len = buffer1.size(); + for(int i = 0; i < len; i++ ) { + if (arr[i] != arrOther[i]) { + return arr[i] - arrOther[i]; + } + } + return 0; + } catch (IOException e) { + throw new RuntimeException("Could not compare reference.", e); + } + } + + @Override + public boolean supportsNormalizedKey() { + return true; + } + + @Override + public boolean supportsSerializationWithKeyNormalization() { + return false; + } + + @Override + public int getNormalizeKeyLen() { + return Integer.MAX_VALUE; + } + + @Override + public boolean isNormalizedKeyPrefixOnly(int keyBytes) { + return true; + } + + @Override + public void putNormalizedKey(KV record, MemorySegment target, int offset, int numBytes) { + buffer1.reset(); + try { + keyCoder.encode(record.getKey(), buffer1, Coder.Context.NESTED); + } catch (IOException e) { + throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e); + } + final byte[] data = buffer1.getBuffer(); + final int limit = offset + numBytes; + + int numBytesPut = Math.min(numBytes, buffer1.size()); + + target.put(offset, data, 0, numBytesPut); + + offset += numBytesPut; + + while (offset < limit) { + target.put(offset++, (byte) 0); + } + } + + @Override + public void writeWithKeyNormalization(KV record, DataOutputView target) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public KV readWithKeyDenormalization(KV reuse, DataInputView source) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean invertNormalizedKey() { + return false; + } + + @Override + public TypeComparator> duplicate() { + return new KvCoderComperator<>(coder); + } + + @Override + public int extractKeys(Object record, Object[] target, int index) { + KV kv = (KV) record; + K k = kv.getKey(); + target[index] = k; + return 1; + } + + @Override + public TypeComparator[] getFlatComparators() { + return new TypeComparator[] {new CoderComparator<>(keyCoder)}; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java index 090f79daf7d7..be119182bb3a 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java @@ -34,153 +34,153 @@ */ public class KvCoderTypeInformation extends CompositeType> { - private KvCoder coder; - - // We don't have the Class, so we have to pass null here. What a shame... - private static Object DUMMY = new Object(); - - @SuppressWarnings("unchecked") - public KvCoderTypeInformation(KvCoder coder) { - super(((Class>) DUMMY.getClass())); - this.coder = coder; - Preconditions.checkNotNull(coder); - } - - @Override - @SuppressWarnings("unchecked") - public TypeComparator> createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) { - return new KvCoderComperator((KvCoder) coder); - } - - @Override - public boolean isBasicType() { - return false; - } - - @Override - public boolean isTupleType() { - return false; - } - - @Override - public int getArity() { - return 2; - } - - @Override - @SuppressWarnings("unchecked") - public Class> getTypeClass() { - return privateGetTypeClass(); - } - - @SuppressWarnings("unchecked") - private static Class privateGetTypeClass() { - return (Class) Object.class; - } - - @Override - public boolean isKeyType() { - return true; - } - - @Override - @SuppressWarnings("unchecked") - public TypeSerializer> createSerializer(ExecutionConfig config) { - return new CoderTypeSerializer<>(coder); - } - - @Override - public int getTotalFields() { - return 2; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - KvCoderTypeInformation that = (KvCoderTypeInformation) o; - - return coder.equals(that.coder); - - } - - @Override - public int hashCode() { - return coder.hashCode(); - } - - @Override - public String toString() { - return "CoderTypeInformation{" + - "coder=" + coder + - '}'; - } - - @Override - @SuppressWarnings("unchecked") - public TypeInformation getTypeAt(int pos) { - if (pos == 0) { - return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); - } else if (pos == 1) { - return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); - } else { - throw new RuntimeException("Invalid field position " + pos); - } - } - - @Override - @SuppressWarnings("unchecked") - public TypeInformation getTypeAt(String fieldExpression) { - switch (fieldExpression) { - case "key": - return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); - case "value": - return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); - default: - throw new UnsupportedOperationException("Only KvCoder has fields."); - } - } - - @Override - public String[] getFieldNames() { - return new String[]{"key", "value"}; - } - - @Override - public int getFieldIndex(String fieldName) { - switch (fieldName) { - case "key": - return 0; - case "value": - return 1; - default: - return -1; - } - } - - @Override - public void getFlatFields(String fieldExpression, int offset, List result) { - CoderTypeInformation keyTypeInfo = new CoderTypeInformation<>(coder.getKeyCoder()); - result.add(new FlatFieldDescriptor(0, keyTypeInfo)); - } - - @Override - protected TypeComparatorBuilder> createTypeComparatorBuilder() { - return new KvCoderTypeComparatorBuilder(); - } - - private class KvCoderTypeComparatorBuilder implements TypeComparatorBuilder> { - - @Override - public void initializeTypeComparatorBuilder(int size) {} - - @Override - public void addComparatorField(int fieldId, TypeComparator comparator) {} - - @Override - public TypeComparator> createTypeComparator(ExecutionConfig config) { - return new KvCoderComperator<>(coder); - } - } + private KvCoder coder; + + // We don't have the Class, so we have to pass null here. What a shame... + private static Object DUMMY = new Object(); + + @SuppressWarnings("unchecked") + public KvCoderTypeInformation(KvCoder coder) { + super(((Class>) DUMMY.getClass())); + this.coder = coder; + Preconditions.checkNotNull(coder); + } + + @Override + @SuppressWarnings("unchecked") + public TypeComparator> createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) { + return new KvCoderComperator((KvCoder) coder); + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 2; + } + + @Override + @SuppressWarnings("unchecked") + public Class> getTypeClass() { + return privateGetTypeClass(); + } + + @SuppressWarnings("unchecked") + private static Class privateGetTypeClass() { + return (Class) Object.class; + } + + @Override + public boolean isKeyType() { + return true; + } + + @Override + @SuppressWarnings("unchecked") + public TypeSerializer> createSerializer(ExecutionConfig config) { + return new CoderTypeSerializer<>(coder); + } + + @Override + public int getTotalFields() { + return 2; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + KvCoderTypeInformation that = (KvCoderTypeInformation) o; + + return coder.equals(that.coder); + + } + + @Override + public int hashCode() { + return coder.hashCode(); + } + + @Override + public String toString() { + return "CoderTypeInformation{" + + "coder=" + coder + + '}'; + } + + @Override + @SuppressWarnings("unchecked") + public TypeInformation getTypeAt(int pos) { + if (pos == 0) { + return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); + } else if (pos == 1) { + return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); + } else { + throw new RuntimeException("Invalid field position " + pos); + } + } + + @Override + @SuppressWarnings("unchecked") + public TypeInformation getTypeAt(String fieldExpression) { + switch (fieldExpression) { + case "key": + return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); + case "value": + return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); + default: + throw new UnsupportedOperationException("Only KvCoder has fields."); + } + } + + @Override + public String[] getFieldNames() { + return new String[]{"key", "value"}; + } + + @Override + public int getFieldIndex(String fieldName) { + switch (fieldName) { + case "key": + return 0; + case "value": + return 1; + default: + return -1; + } + } + + @Override + public void getFlatFields(String fieldExpression, int offset, List result) { + CoderTypeInformation keyTypeInfo = new CoderTypeInformation<>(coder.getKeyCoder()); + result.add(new FlatFieldDescriptor(0, keyTypeInfo)); + } + + @Override + protected TypeComparatorBuilder> createTypeComparatorBuilder() { + return new KvCoderTypeComparatorBuilder(); + } + + private class KvCoderTypeComparatorBuilder implements TypeComparatorBuilder> { + + @Override + public void initializeTypeComparatorBuilder(int size) {} + + @Override + public void addComparatorField(int fieldId, TypeComparator comparator) {} + + @Override + public TypeComparator> createTypeComparator(ExecutionConfig config) { + return new KvCoderComperator<>(coder); + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java index 7ce484ac10d2..190d898cfe5d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java @@ -31,82 +31,82 @@ */ public class VoidCoderTypeSerializer extends TypeSerializer { - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public VoidCoderTypeSerializer duplicate() { - return this; - } - - @Override - public VoidValue createInstance() { - return VoidValue.INSTANCE; - } - - @Override - public VoidValue copy(VoidValue from) { - return from; - } - - @Override - public VoidValue copy(VoidValue from, VoidValue reuse) { - return from; - } - - @Override - public int getLength() { - return 0; - } - - @Override - public void serialize(VoidValue record, DataOutputView target) throws IOException { - target.writeByte(1); - } - - @Override - public VoidValue deserialize(DataInputView source) throws IOException { - source.readByte(); - return VoidValue.INSTANCE; - } - - @Override - public VoidValue deserialize(VoidValue reuse, DataInputView source) throws IOException { - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - source.readByte(); - target.writeByte(1); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof VoidCoderTypeSerializer) { - VoidCoderTypeSerializer other = (VoidCoderTypeSerializer) obj; - return other.canEqual(this); - } else { - return false; - } - } - - @Override - public boolean canEqual(Object obj) { - return obj instanceof VoidCoderTypeSerializer; - } - - @Override - public int hashCode() { - return 0; - } - - public static class VoidValue { - private VoidValue() {} - - public static VoidValue INSTANCE = new VoidValue(); - } + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public VoidCoderTypeSerializer duplicate() { + return this; + } + + @Override + public VoidValue createInstance() { + return VoidValue.INSTANCE; + } + + @Override + public VoidValue copy(VoidValue from) { + return from; + } + + @Override + public VoidValue copy(VoidValue from, VoidValue reuse) { + return from; + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(VoidValue record, DataOutputView target) throws IOException { + target.writeByte(1); + } + + @Override + public VoidValue deserialize(DataInputView source) throws IOException { + source.readByte(); + return VoidValue.INSTANCE; + } + + @Override + public VoidValue deserialize(VoidValue reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + source.readByte(); + target.writeByte(1); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof VoidCoderTypeSerializer) { + VoidCoderTypeSerializer other = (VoidCoderTypeSerializer) obj; + return other.canEqual(this); + } else { + return false; + } + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof VoidCoderTypeSerializer; + } + + @Override + public int hashCode() { + return 0; + } + + public static class VoidValue { + private VoidValue() {} + + public static VoidValue INSTANCE = new VoidValue(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java index 924b29769f0f..8f6d67cbae5c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java @@ -31,62 +31,62 @@ * operation. */ public class CombineFnAggregatorWrapper implements Aggregator, Accumulator { - - private AA aa; - private Combine.CombineFn combiner; + + private AA aa; + private Combine.CombineFn combiner; - public CombineFnAggregatorWrapper() { - } + public CombineFnAggregatorWrapper() { + } - public CombineFnAggregatorWrapper(Combine.CombineFn combiner) { - this.combiner = combiner; - this.aa = combiner.createAccumulator(); - } + public CombineFnAggregatorWrapper(Combine.CombineFn combiner) { + this.combiner = combiner; + this.aa = combiner.createAccumulator(); + } - @Override - public void add(AI value) { - combiner.addInput(aa, value); - } + @Override + public void add(AI value) { + combiner.addInput(aa, value); + } - @Override - public Serializable getLocalValue() { - return (Serializable) combiner.extractOutput(aa); - } + @Override + public Serializable getLocalValue() { + return (Serializable) combiner.extractOutput(aa); + } - @Override - public void resetLocal() { - aa = combiner.createAccumulator(); - } + @Override + public void resetLocal() { + aa = combiner.createAccumulator(); + } - @Override - @SuppressWarnings("unchecked") - public void merge(Accumulator other) { - aa = combiner.mergeAccumulators(Lists.newArrayList(aa, ((CombineFnAggregatorWrapper)other).aa)); - } + @Override + @SuppressWarnings("unchecked") + public void merge(Accumulator other) { + aa = combiner.mergeAccumulators(Lists.newArrayList(aa, ((CombineFnAggregatorWrapper)other).aa)); + } - @Override - public Accumulator clone() { - // copy it by merging - AA aaCopy = combiner.mergeAccumulators(Lists.newArrayList(aa)); - CombineFnAggregatorWrapper result = new - CombineFnAggregatorWrapper<>(combiner); - result.aa = aaCopy; - return result; - } + @Override + public Accumulator clone() { + // copy it by merging + AA aaCopy = combiner.mergeAccumulators(Lists.newArrayList(aa)); + CombineFnAggregatorWrapper result = new + CombineFnAggregatorWrapper<>(combiner); + result.aa = aaCopy; + return result; + } - @Override - public void addValue(AI value) { - add(value); - } + @Override + public void addValue(AI value) { + add(value); + } - @Override - public String getName() { - return "CombineFn: " + combiner.toString(); - } + @Override + public String getName() { + return "CombineFn: " + combiner.toString(); + } - @Override - public Combine.CombineFn getCombineFn() { - return combiner; - } + @Override + public Combine.CombineFn getCombineFn() { + return combiner; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java index 90582b06693d..3c9693949be2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java @@ -31,29 +31,29 @@ */ public class DataInputViewWrapper extends InputStream { - private DataInputView inputView; - - public DataInputViewWrapper(DataInputView inputView) { - this.inputView = inputView; - } - - public void setInputView(DataInputView inputView) { - this.inputView = inputView; - } - - @Override - public int read() throws IOException { - try { - return inputView.readUnsignedByte(); - } catch (EOFException e) { - // translate between DataInput and InputStream, - // DataInput signals EOF by exception, InputStream does it by returning -1 - return -1; - } - } - - @Override - public int read(byte[] b, int off, int len) throws IOException { - return inputView.read(b, off, len); - } + private DataInputView inputView; + + public DataInputViewWrapper(DataInputView inputView) { + this.inputView = inputView; + } + + public void setInputView(DataInputView inputView) { + this.inputView = inputView; + } + + @Override + public int read() throws IOException { + try { + return inputView.readUnsignedByte(); + } catch (EOFException e) { + // translate between DataInput and InputStream, + // DataInput signals EOF by exception, InputStream does it by returning -1 + return -1; + } + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + return inputView.read(b, off, len); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java index 46df8e52defa..a222cdd93392 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java @@ -29,24 +29,24 @@ * {@link java.io.OutputStream}. */ public class DataOutputViewWrapper extends OutputStream { - - private DataOutputView outputView; + + private DataOutputView outputView; - public DataOutputViewWrapper(DataOutputView outputView) { - this.outputView = outputView; - } + public DataOutputViewWrapper(DataOutputView outputView) { + this.outputView = outputView; + } - public void setOutputView(DataOutputView outputView) { - this.outputView = outputView; - } + public void setOutputView(DataOutputView outputView) { + this.outputView = outputView; + } - @Override - public void write(int b) throws IOException { - outputView.write(b); - } + @Override + public void write(int b) throws IOException { + outputView.write(b); + } - @Override - public void write(byte[] b, int off, int len) throws IOException { - outputView.write(b, off, len); - } + @Override + public void write(byte[] b, int off, int len) throws IOException { + outputView.write(b, off, len); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java index 1c0dae401ee6..c193a4dedca8 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -33,59 +33,59 @@ */ public class SerializableFnAggregatorWrapper implements Aggregator, Accumulator { - private AO aa; - private Combine.CombineFn combiner; + private AO aa; + private Combine.CombineFn combiner; - public SerializableFnAggregatorWrapper(Combine.CombineFn combiner) { - this.combiner = combiner; - resetLocal(); - } - - @Override - @SuppressWarnings("unchecked") - public void add(AI value) { - this.aa = combiner.apply(ImmutableList.of((AI) aa, value)); - } + public SerializableFnAggregatorWrapper(Combine.CombineFn combiner) { + this.combiner = combiner; + resetLocal(); + } + + @Override + @SuppressWarnings("unchecked") + public void add(AI value) { + this.aa = combiner.apply(ImmutableList.of((AI) aa, value)); + } - @Override - public Serializable getLocalValue() { - return (Serializable) aa; - } + @Override + public Serializable getLocalValue() { + return (Serializable) aa; + } - @Override - public void resetLocal() { - this.aa = combiner.apply(ImmutableList.of()); - } + @Override + public void resetLocal() { + this.aa = combiner.apply(ImmutableList.of()); + } - @Override - @SuppressWarnings("unchecked") - public void merge(Accumulator other) { - this.aa = combiner.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue())); - } + @Override + @SuppressWarnings("unchecked") + public void merge(Accumulator other) { + this.aa = combiner.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue())); + } - @Override - public void addValue(AI value) { - add(value); - } + @Override + public void addValue(AI value) { + add(value); + } - @Override - public String getName() { - return "Aggregator :" + combiner.toString(); - } + @Override + public String getName() { + return "Aggregator :" + combiner.toString(); + } - @Override - public Combine.CombineFn getCombineFn() { - return combiner; - } + @Override + public Combine.CombineFn getCombineFn() { + return combiner; + } - @Override - public Accumulator clone() { - // copy it by merging - AO resultCopy = combiner.apply(Lists.newArrayList((AI) aa)); - SerializableFnAggregatorWrapper result = new - SerializableFnAggregatorWrapper<>(combiner); + @Override + public Accumulator clone() { + // copy it by merging + AO resultCopy = combiner.apply(Lists.newArrayList((AI) aa)); + SerializableFnAggregatorWrapper result = new + SerializableFnAggregatorWrapper<>(combiner); - result.aa = resultCopy; - return result; - } + result.aa = resultCopy; + return result; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java index 8be9abf35a39..3f28c1630b77 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java @@ -38,84 +38,84 @@ */ public class SinkOutputFormat implements OutputFormat { - private final Sink sink; - - private transient PipelineOptions pipelineOptions; - - private Sink.WriteOperation writeOperation; - private Sink.Writer writer; - - private AbstractID uid = new AbstractID(); - - public SinkOutputFormat(Write.Bound transform, PipelineOptions pipelineOptions) { - this.sink = extractSink(transform); - this.pipelineOptions = Preconditions.checkNotNull(pipelineOptions); - } - - private Sink extractSink(Write.Bound transform) { - // TODO possibly add a getter in the upstream - try { - Field sinkField = transform.getClass().getDeclaredField("sink"); - sinkField.setAccessible(true); - @SuppressWarnings("unchecked") - Sink extractedSink = (Sink) sinkField.get(transform); - return extractedSink; - } catch (NoSuchFieldException | IllegalAccessException e) { - throw new RuntimeException("Could not acquire custom sink field.", e); - } - } - - @Override - public void configure(Configuration configuration) { - writeOperation = sink.createWriteOperation(pipelineOptions); - try { - writeOperation.initialize(pipelineOptions); - } catch (Exception e) { - throw new RuntimeException("Failed to initialize the write operation.", e); - } - } - - @Override - public void open(int taskNumber, int numTasks) throws IOException { - try { - writer = writeOperation.createWriter(pipelineOptions); - } catch (Exception e) { - throw new IOException("Couldn't create writer.", e); - } - try { - writer.open(uid + "-" + String.valueOf(taskNumber)); - } catch (Exception e) { - throw new IOException("Couldn't open writer.", e); - } - } - - @Override - public void writeRecord(T record) throws IOException { - try { - writer.write(record); - } catch (Exception e) { - throw new IOException("Couldn't write record.", e); - } - } - - @Override - public void close() throws IOException { - try { - writer.close(); - } catch (Exception e) { - throw new IOException("Couldn't close writer.", e); - } - } - - private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException { - out.defaultWriteObject(); - ObjectMapper mapper = new ObjectMapper(); - mapper.writeValue(out, pipelineOptions); - } - - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - ObjectMapper mapper = new ObjectMapper(); - pipelineOptions = mapper.readValue(in, PipelineOptions.class); - } + private final Sink sink; + + private transient PipelineOptions pipelineOptions; + + private Sink.WriteOperation writeOperation; + private Sink.Writer writer; + + private AbstractID uid = new AbstractID(); + + public SinkOutputFormat(Write.Bound transform, PipelineOptions pipelineOptions) { + this.sink = extractSink(transform); + this.pipelineOptions = Preconditions.checkNotNull(pipelineOptions); + } + + private Sink extractSink(Write.Bound transform) { + // TODO possibly add a getter in the upstream + try { + Field sinkField = transform.getClass().getDeclaredField("sink"); + sinkField.setAccessible(true); + @SuppressWarnings("unchecked") + Sink extractedSink = (Sink) sinkField.get(transform); + return extractedSink; + } catch (NoSuchFieldException | IllegalAccessException e) { + throw new RuntimeException("Could not acquire custom sink field.", e); + } + } + + @Override + public void configure(Configuration configuration) { + writeOperation = sink.createWriteOperation(pipelineOptions); + try { + writeOperation.initialize(pipelineOptions); + } catch (Exception e) { + throw new RuntimeException("Failed to initialize the write operation.", e); + } + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + try { + writer = writeOperation.createWriter(pipelineOptions); + } catch (Exception e) { + throw new IOException("Couldn't create writer.", e); + } + try { + writer.open(uid + "-" + String.valueOf(taskNumber)); + } catch (Exception e) { + throw new IOException("Couldn't open writer.", e); + } + } + + @Override + public void writeRecord(T record) throws IOException { + try { + writer.write(record); + } catch (Exception e) { + throw new IOException("Couldn't write record.", e); + } + } + + @Override + public void close() throws IOException { + try { + writer.close(); + } catch (Exception e) { + throw new IOException("Couldn't close writer.", e); + } + } + + private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException { + out.defaultWriteObject(); + ObjectMapper mapper = new ObjectMapper(); + mapper.writeValue(out, pipelineOptions); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + ObjectMapper mapper = new ObjectMapper(); + pipelineOptions = mapper.readValue(in, PipelineOptions.class); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java index 64dc07267e7c..598161842e5c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java @@ -41,124 +41,124 @@ * Dataflow {@link com.google.cloud.dataflow.sdk.io.Source}. */ public class SourceInputFormat implements InputFormat> { - private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class); - - private final BoundedSource initialSource; - private transient PipelineOptions options; - - private BoundedSource.BoundedReader reader = null; - private boolean reachedEnd = true; - - public SourceInputFormat(BoundedSource initialSource, PipelineOptions options) { - this.initialSource = initialSource; - this.options = options; - } - - private void writeObject(ObjectOutputStream out) - throws IOException, ClassNotFoundException { - out.defaultWriteObject(); - ObjectMapper mapper = new ObjectMapper(); - mapper.writeValue(out, options); - } - - private void readObject(ObjectInputStream in) - throws IOException, ClassNotFoundException { - in.defaultReadObject(); - ObjectMapper mapper = new ObjectMapper(); - options = mapper.readValue(in, PipelineOptions.class); - } - - @Override - public void configure(Configuration configuration) {} - - @Override - public void open(SourceInputSplit sourceInputSplit) throws IOException { - reader = ((BoundedSource) sourceInputSplit.getSource()).createReader(options); - reachedEnd = false; - } - - @Override - public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException { - try { - final long estimatedSize = initialSource.getEstimatedSizeBytes(options); - - return new BaseStatistics() { - @Override - public long getTotalInputSize() { - return estimatedSize; - - } - - @Override - public long getNumberOfRecords() { - return BaseStatistics.NUM_RECORDS_UNKNOWN; - } - - @Override - public float getAverageRecordWidth() { - return BaseStatistics.AVG_RECORD_BYTES_UNKNOWN; - } - }; - } catch (Exception e) { - LOG.warn("Could not read Source statistics: {}", e); - } - - return null; - } - - @Override - @SuppressWarnings("unchecked") - public SourceInputSplit[] createInputSplits(int numSplits) throws IOException { - long desiredSizeBytes; - try { - desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits; - List> shards = initialSource.splitIntoBundles(desiredSizeBytes, - options); - List> splits = new ArrayList<>(); - int splitCount = 0; - for (Source shard: shards) { - splits.add(new SourceInputSplit<>(shard, splitCount++)); - } - return splits.toArray(new SourceInputSplit[splits.size()]); - } catch (Exception e) { - throw new IOException("Could not create input splits from Source.", e); - } - } - - @Override - public InputSplitAssigner getInputSplitAssigner(final SourceInputSplit[] sourceInputSplits) { - return new InputSplitAssigner() { - private int index = 0; - private final SourceInputSplit[] splits = sourceInputSplits; - @Override - public InputSplit getNextInputSplit(String host, int taskId) { - if (index < splits.length) { - return splits[index++]; - } else { - return null; - } - } - }; - } - - - @Override - public boolean reachedEnd() throws IOException { - return reachedEnd; - } - - @Override - public T nextRecord(T t) throws IOException { - - reachedEnd = !reader.advance(); - if (!reachedEnd) { - return reader.getCurrent(); - } - return null; - } - - @Override - public void close() throws IOException { - reader.close(); - } + private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class); + + private final BoundedSource initialSource; + private transient PipelineOptions options; + + private BoundedSource.BoundedReader reader = null; + private boolean reachedEnd = true; + + public SourceInputFormat(BoundedSource initialSource, PipelineOptions options) { + this.initialSource = initialSource; + this.options = options; + } + + private void writeObject(ObjectOutputStream out) + throws IOException, ClassNotFoundException { + out.defaultWriteObject(); + ObjectMapper mapper = new ObjectMapper(); + mapper.writeValue(out, options); + } + + private void readObject(ObjectInputStream in) + throws IOException, ClassNotFoundException { + in.defaultReadObject(); + ObjectMapper mapper = new ObjectMapper(); + options = mapper.readValue(in, PipelineOptions.class); + } + + @Override + public void configure(Configuration configuration) {} + + @Override + public void open(SourceInputSplit sourceInputSplit) throws IOException { + reader = ((BoundedSource) sourceInputSplit.getSource()).createReader(options); + reachedEnd = false; + } + + @Override + public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException { + try { + final long estimatedSize = initialSource.getEstimatedSizeBytes(options); + + return new BaseStatistics() { + @Override + public long getTotalInputSize() { + return estimatedSize; + + } + + @Override + public long getNumberOfRecords() { + return BaseStatistics.NUM_RECORDS_UNKNOWN; + } + + @Override + public float getAverageRecordWidth() { + return BaseStatistics.AVG_RECORD_BYTES_UNKNOWN; + } + }; + } catch (Exception e) { + LOG.warn("Could not read Source statistics: {}", e); + } + + return null; + } + + @Override + @SuppressWarnings("unchecked") + public SourceInputSplit[] createInputSplits(int numSplits) throws IOException { + long desiredSizeBytes; + try { + desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits; + List> shards = initialSource.splitIntoBundles(desiredSizeBytes, + options); + List> splits = new ArrayList<>(); + int splitCount = 0; + for (Source shard: shards) { + splits.add(new SourceInputSplit<>(shard, splitCount++)); + } + return splits.toArray(new SourceInputSplit[splits.size()]); + } catch (Exception e) { + throw new IOException("Could not create input splits from Source.", e); + } + } + + @Override + public InputSplitAssigner getInputSplitAssigner(final SourceInputSplit[] sourceInputSplits) { + return new InputSplitAssigner() { + private int index = 0; + private final SourceInputSplit[] splits = sourceInputSplits; + @Override + public InputSplit getNextInputSplit(String host, int taskId) { + if (index < splits.length) { + return splits[index++]; + } else { + return null; + } + } + }; + } + + + @Override + public boolean reachedEnd() throws IOException { + return reachedEnd; + } + + @Override + public T nextRecord(T t) throws IOException { + + reachedEnd = !reader.advance(); + if (!reachedEnd) { + return reader.getCurrent(); + } + return null; + } + + @Override + public void close() throws IOException { + reader.close(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java index 2b93ab78e65c..86fdadafa8fa 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java @@ -29,24 +29,24 @@ */ public class SourceInputSplit implements InputSplit { - private Source source; - private int splitNumber; + private Source source; + private int splitNumber; - public SourceInputSplit() { - } + public SourceInputSplit() { + } - public SourceInputSplit(Source source, int splitNumber) { - this.source = source; - this.splitNumber = splitNumber; - } + public SourceInputSplit(Source source, int splitNumber) { + this.source = source; + this.splitNumber = splitNumber; + } - @Override - public int getSplitNumber() { - return splitNumber; - } + @Override + public int getSplitNumber() { + return splitNumber; + } - public Source getSource() { - return source; - } + public Source getSource() { + return source; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index eabc307d696e..7dae0b0443c5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -44,221 +44,221 @@ * */ public abstract class FlinkAbstractParDoWrapper extends RichFlatMapFunction, WindowedValue> { - private final DoFn doFn; - private final WindowingStrategy windowingStrategy; - private transient PipelineOptions options; - - private DoFnProcessContext context; - - public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy windowingStrategy, DoFn doFn) { - Preconditions.checkNotNull(options); - Preconditions.checkNotNull(windowingStrategy); - Preconditions.checkNotNull(doFn); - - this.doFn = doFn; - this.options = options; - this.windowingStrategy = windowingStrategy; - } - - private void initContext(DoFn function, Collector> outCollector) { - if (this.context == null) { - this.context = new DoFnProcessContext(function, outCollector); - } - } - - @Override - public void flatMap(WindowedValue value, Collector> out) throws Exception { - this.initContext(doFn, out); - - // for each window the element belongs to, create a new copy here. - Collection windows = value.getWindows(); - if (windows.size() <= 1) { - processElement(value); - } else { - for (BoundedWindow window : windows) { - processElement(WindowedValue.of( - value.getValue(), value.getTimestamp(), window, value.getPane())); - } - } - } - - private void processElement(WindowedValue value) throws Exception { - this.context.setElement(value); - this.doFn.startBundle(context); - doFn.processElement(context); - this.doFn.finishBundle(context); - } - - private class DoFnProcessContext extends DoFn.ProcessContext { - - private final DoFn fn; - - protected final Collector> collector; - - private WindowedValue element; - - private DoFnProcessContext(DoFn function, Collector> outCollector) { - function.super(); - super.setupDelegateAggregators(); - - this.fn = function; - this.collector = outCollector; - } - - public void setElement(WindowedValue value) { - this.element = value; - } - - @Override - public IN element() { - return this.element.getValue(); - } - - @Override - public Instant timestamp() { - return this.element.getTimestamp(); - } - - @Override - public BoundedWindow window() { - if (!(fn instanceof DoFn.RequiresWindowAccess)) { - throw new UnsupportedOperationException( - "window() is only available in the context of a DoFn marked as RequiresWindow."); - } - - Collection windows = this.element.getWindows(); - if (windows.size() != 1) { - throw new IllegalArgumentException("Each element is expected to belong to 1 window. " + - "This belongs to " + windows.size() + "."); - } - return windows.iterator().next(); - } - - @Override - public PaneInfo pane() { - return this.element.getPane(); - } - - @Override - public WindowingInternals windowingInternals() { - return windowingInternalsHelper(element, collector); - } - - @Override - public PipelineOptions getPipelineOptions() { - return options; - } - - @Override - public T sideInput(PCollectionView view) { - throw new RuntimeException("sideInput() is not supported in Streaming mode."); - } - - @Override - public void output(OUTDF output) { - outputWithTimestamp(output, this.element.getTimestamp()); - } - - @Override - public void outputWithTimestamp(OUTDF output, Instant timestamp) { - outputWithTimestampHelper(element, output, timestamp, collector); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - sideOutputWithTimestamp(tag, output, this.element.getTimestamp()); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - sideOutputWithTimestampHelper(element, output, timestamp, collector, tag); - } - - @Override - protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { - Accumulator acc = getRuntimeContext().getAccumulator(name); - if (acc != null) { - AccumulatorHelper.compareAccumulatorTypes(name, - SerializableFnAggregatorWrapper.class, acc.getClass()); - return (Aggregator) acc; - } - - SerializableFnAggregatorWrapper accumulator = - new SerializableFnAggregatorWrapper<>(combiner); - getRuntimeContext().addAccumulator(name, accumulator); - return accumulator; - } - } - - protected void checkTimestamp(WindowedValue ref, Instant timestamp) { - if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) { - throw new IllegalArgumentException(String.format( - "Cannot output with timestamp %s. Output timestamps must be no earlier than the " - + "timestamp of the current input (%s) minus the allowed skew (%s). See the " - + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.", - timestamp, ref.getTimestamp(), - PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod()))); - } - } - - protected WindowedValue makeWindowedValue( - T output, Instant timestamp, Collection windows, PaneInfo pane) { - final Instant inputTimestamp = timestamp; - final WindowFn windowFn = windowingStrategy.getWindowFn(); - - if (timestamp == null) { - timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; - } - - if (windows == null) { - try { - windows = windowFn.assignWindows(windowFn.new AssignContext() { - @Override - public Object element() { - throw new UnsupportedOperationException( - "WindowFn attempted to access input element when none was available"); - } - - @Override - public Instant timestamp() { - if (inputTimestamp == null) { - throw new UnsupportedOperationException( - "WindowFn attempted to access input timestamp when none was available"); - } - return inputTimestamp; - } - - @Override - public Collection windows() { - throw new UnsupportedOperationException( - "WindowFn attempted to access input windows when none were available"); - } - }); - } catch (Exception e) { - throw UserCodeException.wrap(e); - } - } - - return WindowedValue.of(output, timestamp, windows, pane); - } - - /////////// ABSTRACT METHODS TO BE IMPLEMENTED BY SUBCLASSES ///////////////// - - public abstract void outputWithTimestampHelper( - WindowedValue inElement, - OUTDF output, - Instant timestamp, - Collector> outCollector); - - public abstract void sideOutputWithTimestampHelper( - WindowedValue inElement, - T output, - Instant timestamp, - Collector> outCollector, - TupleTag tag); - - public abstract WindowingInternals windowingInternalsHelper( - WindowedValue inElement, - Collector> outCollector); + private final DoFn doFn; + private final WindowingStrategy windowingStrategy; + private transient PipelineOptions options; + + private DoFnProcessContext context; + + public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy windowingStrategy, DoFn doFn) { + Preconditions.checkNotNull(options); + Preconditions.checkNotNull(windowingStrategy); + Preconditions.checkNotNull(doFn); + + this.doFn = doFn; + this.options = options; + this.windowingStrategy = windowingStrategy; + } + + private void initContext(DoFn function, Collector> outCollector) { + if (this.context == null) { + this.context = new DoFnProcessContext(function, outCollector); + } + } + + @Override + public void flatMap(WindowedValue value, Collector> out) throws Exception { + this.initContext(doFn, out); + + // for each window the element belongs to, create a new copy here. + Collection windows = value.getWindows(); + if (windows.size() <= 1) { + processElement(value); + } else { + for (BoundedWindow window : windows) { + processElement(WindowedValue.of( + value.getValue(), value.getTimestamp(), window, value.getPane())); + } + } + } + + private void processElement(WindowedValue value) throws Exception { + this.context.setElement(value); + this.doFn.startBundle(context); + doFn.processElement(context); + this.doFn.finishBundle(context); + } + + private class DoFnProcessContext extends DoFn.ProcessContext { + + private final DoFn fn; + + protected final Collector> collector; + + private WindowedValue element; + + private DoFnProcessContext(DoFn function, Collector> outCollector) { + function.super(); + super.setupDelegateAggregators(); + + this.fn = function; + this.collector = outCollector; + } + + public void setElement(WindowedValue value) { + this.element = value; + } + + @Override + public IN element() { + return this.element.getValue(); + } + + @Override + public Instant timestamp() { + return this.element.getTimestamp(); + } + + @Override + public BoundedWindow window() { + if (!(fn instanceof DoFn.RequiresWindowAccess)) { + throw new UnsupportedOperationException( + "window() is only available in the context of a DoFn marked as RequiresWindow."); + } + + Collection windows = this.element.getWindows(); + if (windows.size() != 1) { + throw new IllegalArgumentException("Each element is expected to belong to 1 window. " + + "This belongs to " + windows.size() + "."); + } + return windows.iterator().next(); + } + + @Override + public PaneInfo pane() { + return this.element.getPane(); + } + + @Override + public WindowingInternals windowingInternals() { + return windowingInternalsHelper(element, collector); + } + + @Override + public PipelineOptions getPipelineOptions() { + return options; + } + + @Override + public T sideInput(PCollectionView view) { + throw new RuntimeException("sideInput() is not supported in Streaming mode."); + } + + @Override + public void output(OUTDF output) { + outputWithTimestamp(output, this.element.getTimestamp()); + } + + @Override + public void outputWithTimestamp(OUTDF output, Instant timestamp) { + outputWithTimestampHelper(element, output, timestamp, collector); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + sideOutputWithTimestamp(tag, output, this.element.getTimestamp()); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + sideOutputWithTimestampHelper(element, output, timestamp, collector, tag); + } + + @Override + protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { + Accumulator acc = getRuntimeContext().getAccumulator(name); + if (acc != null) { + AccumulatorHelper.compareAccumulatorTypes(name, + SerializableFnAggregatorWrapper.class, acc.getClass()); + return (Aggregator) acc; + } + + SerializableFnAggregatorWrapper accumulator = + new SerializableFnAggregatorWrapper<>(combiner); + getRuntimeContext().addAccumulator(name, accumulator); + return accumulator; + } + } + + protected void checkTimestamp(WindowedValue ref, Instant timestamp) { + if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) { + throw new IllegalArgumentException(String.format( + "Cannot output with timestamp %s. Output timestamps must be no earlier than the " + + "timestamp of the current input (%s) minus the allowed skew (%s). See the " + + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.", + timestamp, ref.getTimestamp(), + PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod()))); + } + } + + protected WindowedValue makeWindowedValue( + T output, Instant timestamp, Collection windows, PaneInfo pane) { + final Instant inputTimestamp = timestamp; + final WindowFn windowFn = windowingStrategy.getWindowFn(); + + if (timestamp == null) { + timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + if (windows == null) { + try { + windows = windowFn.assignWindows(windowFn.new AssignContext() { + @Override + public Object element() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input element when none was available"); + } + + @Override + public Instant timestamp() { + if (inputTimestamp == null) { + throw new UnsupportedOperationException( + "WindowFn attempted to access input timestamp when none was available"); + } + return inputTimestamp; + } + + @Override + public Collection windows() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input windows when none were available"); + } + }); + } catch (Exception e) { + throw UserCodeException.wrap(e); + } + } + + return WindowedValue.of(output, timestamp, windows, pane); + } + + /////////// ABSTRACT METHODS TO BE IMPLEMENTED BY SUBCLASSES ///////////////// + + public abstract void outputWithTimestampHelper( + WindowedValue inElement, + OUTDF output, + Instant timestamp, + Collector> outCollector); + + public abstract void sideOutputWithTimestampHelper( + WindowedValue inElement, + T output, + Instant timestamp, + Collector> outCollector, + TupleTag tag); + + public abstract WindowingInternals windowingInternalsHelper( + WindowedValue inElement, + Collector> outCollector); } \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java index fb3d329990dd..55235c9d4725 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -68,562 +68,562 @@ * for furhter processing. */ public class FlinkGroupAlsoByWindowWrapper - extends AbstractStreamOperator>> - implements OneInputStreamOperator>, WindowedValue>> { - - private static final long serialVersionUID = 1L; - - private transient PipelineOptions options; - - private transient CoderRegistry coderRegistry; - - private DoFn, KV> operator; - - private ProcessContext context; - - private final WindowingStrategy, BoundedWindow> windowingStrategy; - - private final Combine.KeyedCombineFn combineFn; - - private final KvCoder inputKvCoder; - - /** - * State is kept per-key. This data structure keeps this mapping between an active key, i.e. a - * key whose elements are currently waiting to be processed, and its associated state. - */ - private Map> perKeyStateInternals = new HashMap<>(); - - /** - * Timers waiting to be processed. - */ - private Map> activeTimers = new HashMap<>(); - - private FlinkTimerInternals timerInternals = new FlinkTimerInternals(); - - /** - * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy. - * This method assumes that elements are already grouped by key. - *

    - * The difference with {@link #createForIterable(PipelineOptions, PCollection, KeyedStream)} - * is that this method assumes that a combiner function is provided - * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}). - * A combiner helps at increasing the speed and, in most of the cases, reduce the per-window state. - * - * @param options the general job configuration options. - * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}. - * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key. - * @param combiner the combiner to be used. - * @param outputKvCoder the type of the output values. - */ - public static DataStream>> create( - PipelineOptions options, - PCollection input, - KeyedStream>, K> groupedStreamByKey, - Combine.KeyedCombineFn combiner, - KvCoder outputKvCoder) { - Preconditions.checkNotNull(options); - - KvCoder inputKvCoder = (KvCoder) input.getCoder(); - FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper<>(options, - input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, combiner); - - Coder>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of( - outputKvCoder, - input.getWindowingStrategy().getWindowFn().windowCoder()); - - CoderTypeInformation>> outputTypeInfo = - new CoderTypeInformation<>(windowedOutputElemCoder); - - DataStream>> groupedByKeyAndWindow = groupedStreamByKey - .transform("GroupByWindowWithCombiner", - new CoderTypeInformation<>(outputKvCoder), - windower) - .returns(outputTypeInfo); - - return groupedByKeyAndWindow; - } - - /** - * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy. - * This method assumes that elements are already grouped by key. - *

    - * The difference with {@link #create(PipelineOptions, PCollection, KeyedStream, Combine.KeyedCombineFn, KvCoder)} - * is that this method assumes no combiner function - * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}). - * - * @param options the general job configuration options. - * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}. - * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key. - */ - public static DataStream>>> createForIterable( - PipelineOptions options, - PCollection input, - KeyedStream>, K> groupedStreamByKey) { - Preconditions.checkNotNull(options); - - KvCoder inputKvCoder = (KvCoder) input.getCoder(); - Coder keyCoder = inputKvCoder.getKeyCoder(); - Coder inputValueCoder = inputKvCoder.getValueCoder(); - - FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper(options, - input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, null); - - Coder> valueIterCoder = IterableCoder.of(inputValueCoder); - KvCoder> outputElemCoder = KvCoder.of(keyCoder, valueIterCoder); - - Coder>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of( - outputElemCoder, - input.getWindowingStrategy().getWindowFn().windowCoder()); - - CoderTypeInformation>>> outputTypeInfo = - new CoderTypeInformation<>(windowedOutputElemCoder); - - DataStream>>> groupedByKeyAndWindow = groupedStreamByKey - .transform("GroupByWindow", - new CoderTypeInformation<>(windowedOutputElemCoder), - windower) - .returns(outputTypeInfo); - - return groupedByKeyAndWindow; - } - - public static FlinkGroupAlsoByWindowWrapper - createForTesting(PipelineOptions options, - CoderRegistry registry, - WindowingStrategy, BoundedWindow> windowingStrategy, - KvCoder inputCoder, - Combine.KeyedCombineFn combiner) { - Preconditions.checkNotNull(options); - - return new FlinkGroupAlsoByWindowWrapper(options, registry, windowingStrategy, inputCoder, combiner); - } - - private FlinkGroupAlsoByWindowWrapper(PipelineOptions options, - CoderRegistry registry, - WindowingStrategy, BoundedWindow> windowingStrategy, - KvCoder inputCoder, - Combine.KeyedCombineFn combiner) { - Preconditions.checkNotNull(options); - - this.options = Preconditions.checkNotNull(options); - this.coderRegistry = Preconditions.checkNotNull(registry); - this.inputKvCoder = Preconditions.checkNotNull(inputCoder);//(KvCoder) input.getCoder(); - this.windowingStrategy = Preconditions.checkNotNull(windowingStrategy);//input.getWindowingStrategy(); - this.combineFn = combiner; - this.operator = createGroupAlsoByWindowOperator(); - this.chainingStrategy = ChainingStrategy.ALWAYS; - } - - @Override - public void open() throws Exception { - super.open(); - this.context = new ProcessContext(operator, new TimestampedCollector<>(output), this.timerInternals); - } - - /** - * Create the adequate {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn}, - * if not already created. - * If a {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn} was provided, then - * a function with that combiner is created, so that elements are combined as they arrive. This is - * done for speed and (in most of the cases) for reduction of the per-window state. - */ - private DoFn, KV> createGroupAlsoByWindowOperator() { - if (this.operator == null) { - if (this.combineFn == null) { - // Thus VOUT == Iterable - Coder inputValueCoder = inputKvCoder.getValueCoder(); - - this.operator = (DoFn) GroupAlsoByWindowViaWindowSetDoFn.create( - (WindowingStrategy) this.windowingStrategy, SystemReduceFn.buffering(inputValueCoder)); - } else { - Coder inputKeyCoder = inputKvCoder.getKeyCoder(); - - AppliedCombineFn appliedCombineFn = AppliedCombineFn - .withInputCoder(combineFn, coderRegistry, inputKvCoder); - - this.operator = GroupAlsoByWindowViaWindowSetDoFn.create( - (WindowingStrategy) this.windowingStrategy, SystemReduceFn.combining(inputKeyCoder, appliedCombineFn)); - } - } - return this.operator; - } - - private void processKeyedWorkItem(KeyedWorkItem workItem) throws Exception { - context.setElement(workItem, getStateInternalsForKey(workItem.key())); - - // TODO: Ideally startBundle/finishBundle would be called when the operator is first used / about to be discarded. - operator.startBundle(context); - operator.processElement(context); - operator.finishBundle(context); - } - - @Override - public void processElement(StreamRecord>> element) throws Exception { - ArrayList> elements = new ArrayList<>(); - elements.add(WindowedValue.of(element.getValue().getValue().getValue(), element.getValue().getTimestamp(), - element.getValue().getWindows(), element.getValue().getPane())); - processKeyedWorkItem(KeyedWorkItems.elementsWorkItem(element.getValue().getValue().getKey(), elements)); - } - - @Override - public void processWatermark(Watermark mark) throws Exception { - context.setCurrentInputWatermark(new Instant(mark.getTimestamp())); - - Multimap timers = getTimersReadyToProcess(mark.getTimestamp()); - if (!timers.isEmpty()) { - for (K key : timers.keySet()) { - processKeyedWorkItem(KeyedWorkItems.timersWorkItem(key, timers.get(key))); - } - } - - /** - * This is to take into account the different semantics of the Watermark in Flink and - * in Dataflow. To understand the reasoning behind the Dataflow semantics and its - * watermark holding logic, see the documentation of - * {@link WatermarkHold#addHold(ReduceFn.ProcessValueContext, boolean)} - * */ - long millis = Long.MAX_VALUE; - for (FlinkStateInternals state : perKeyStateInternals.values()) { - Instant watermarkHold = state.getWatermarkHold(); - if (watermarkHold != null && watermarkHold.getMillis() < millis) { - millis = watermarkHold.getMillis(); - } - } - - if (mark.getTimestamp() < millis) { - millis = mark.getTimestamp(); - } - - context.setCurrentOutputWatermark(new Instant(millis)); - - // Don't forget to re-emit the watermark for further operators down the line. - // This is critical for jobs with multiple aggregation steps. - // Imagine a job with a groupByKey() on key K1, followed by a map() that changes - // the key K1 to K2, and another groupByKey() on K2. In this case, if the watermark - // is not re-emitted, the second aggregation would never be triggered, and no result - // will be produced. - output.emitWatermark(new Watermark(millis)); - } - - @Override - public void close() throws Exception { - super.close(); - } - - private void registerActiveTimer(K key, TimerInternals.TimerData timer) { - Set timersForKey = activeTimers.get(key); - if (timersForKey == null) { - timersForKey = new HashSet<>(); - } - timersForKey.add(timer); - activeTimers.put(key, timersForKey); - } - - private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) { - Set timersForKey = activeTimers.get(key); - if (timersForKey != null) { - timersForKey.remove(timer); - if (timersForKey.isEmpty()) { - activeTimers.remove(key); - } else { - activeTimers.put(key, timersForKey); - } - } - } - - /** - * Returns the list of timers that are ready to fire. These are the timers - * that are registered to be triggered at a time before the current watermark. - * We keep these timers in a Set, so that they are deduplicated, as the same - * timer can be registered multiple times. - */ - private Multimap getTimersReadyToProcess(long currentWatermark) { - - // we keep the timers to return in a different list and launch them later - // because we cannot prevent a trigger from registering another trigger, - // which would lead to concurrent modification exception. - Multimap toFire = HashMultimap.create(); - - Iterator>> it = activeTimers.entrySet().iterator(); - while (it.hasNext()) { - Map.Entry> keyWithTimers = it.next(); - - Iterator timerIt = keyWithTimers.getValue().iterator(); - while (timerIt.hasNext()) { - TimerInternals.TimerData timerData = timerIt.next(); - if (timerData.getTimestamp().isBefore(currentWatermark)) { - toFire.put(keyWithTimers.getKey(), timerData); - timerIt.remove(); - } - } - - if (keyWithTimers.getValue().isEmpty()) { - it.remove(); - } - } - return toFire; - } - - /** - * Gets the state associated with the specified key. - * - * @param key the key whose state we want. - * @return The {@link FlinkStateInternals} - * associated with that key. - */ - private FlinkStateInternals getStateInternalsForKey(K key) { - FlinkStateInternals stateInternals = perKeyStateInternals.get(key); - if (stateInternals == null) { - Coder windowCoder = this.windowingStrategy.getWindowFn().windowCoder(); - OutputTimeFn outputTimeFn = this.windowingStrategy.getWindowFn().getOutputTimeFn(); - stateInternals = new FlinkStateInternals<>(key, inputKvCoder.getKeyCoder(), windowCoder, outputTimeFn); - perKeyStateInternals.put(key, stateInternals); - } - return stateInternals; - } - - private class FlinkTimerInternals extends AbstractFlinkTimerInternals { - @Override - public void setTimer(TimerData timerKey) { - registerActiveTimer(context.element().key(), timerKey); - } - - @Override - public void deleteTimer(TimerData timerKey) { - unregisterActiveTimer(context.element().key(), timerKey); - } - } - - private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn>.ProcessContext { - - private final FlinkTimerInternals timerInternals; - - private final TimestampedCollector>> collector; - - private FlinkStateInternals stateInternals; - - private KeyedWorkItem element; - - public ProcessContext(DoFn, KV> function, - TimestampedCollector>> outCollector, - FlinkTimerInternals timerInternals) { - function.super(); - super.setupDelegateAggregators(); - - this.collector = Preconditions.checkNotNull(outCollector); - this.timerInternals = Preconditions.checkNotNull(timerInternals); - } - - public void setElement(KeyedWorkItem element, - FlinkStateInternals stateForKey) { - this.element = element; - this.stateInternals = stateForKey; - } - - public void setCurrentInputWatermark(Instant watermark) { - this.timerInternals.setCurrentInputWatermark(watermark); - } - - public void setCurrentOutputWatermark(Instant watermark) { - this.timerInternals.setCurrentOutputWatermark(watermark); - } - - @Override - public KeyedWorkItem element() { - return this.element; - } - - @Override - public Instant timestamp() { - throw new UnsupportedOperationException("timestamp() is not available when processing KeyedWorkItems."); - } - - @Override - public PipelineOptions getPipelineOptions() { - // TODO: PipelineOptions need to be available on the workers. - // Ideally they are captured as part of the pipeline. - // For now, construct empty options so that StateContexts.createFromComponents - // will yield a valid StateContext, which is needed to support the StateContext.window(). - if (options == null) { - options = new PipelineOptions() { - @Override - public T as(Class kls) { - return null; - } - - @Override - public T cloneAs(Class kls) { - return null; - } - - @Override - public Class> getRunner() { - return null; - } - - @Override - public void setRunner(Class> kls) { - - } - - @Override - public CheckEnabled getStableUniqueNames() { - return null; - } - - @Override - public void setStableUniqueNames(CheckEnabled enabled) { - } - }; - } - return options; - } - - @Override - public void output(KV output) { - throw new UnsupportedOperationException( - "output() is not available when processing KeyedWorkItems."); - } - - @Override - public void outputWithTimestamp(KV output, Instant timestamp) { - throw new UnsupportedOperationException( - "outputWithTimestamp() is not available when processing KeyedWorkItems."); - } - - @Override - public PaneInfo pane() { - throw new UnsupportedOperationException("pane() is not available when processing KeyedWorkItems."); - } - - @Override - public BoundedWindow window() { - throw new UnsupportedOperationException( - "window() is not available when processing KeyedWorkItems."); - } - - @Override - public WindowingInternals, KV> windowingInternals() { - return new WindowingInternals, KV>() { - - @Override - public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() { - return stateInternals; - } - - @Override - public void outputWindowedValue(KV output, Instant timestamp, Collection windows, PaneInfo pane) { - // TODO: No need to represent timestamp twice. - collector.setAbsoluteTimestamp(timestamp.getMillis()); - collector.collect(WindowedValue.of(output, timestamp, windows, pane)); - - } - - @Override - public TimerInternals timerInternals() { - return timerInternals; - } - - @Override - public Collection windows() { - throw new UnsupportedOperationException("windows() is not available in Streaming mode."); - } - - @Override - public PaneInfo pane() { - throw new UnsupportedOperationException("pane() is not available in Streaming mode."); - } - - @Override - public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { - throw new RuntimeException("writePCollectionViewData() not available in Streaming mode."); - } - - @Override - public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { - throw new RuntimeException("sideInput() is not available in Streaming mode."); - } - }; - } - - @Override - public T sideInput(PCollectionView view) { - throw new RuntimeException("sideInput() is not supported in Streaming mode."); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - // ignore the side output, this can happen when a user does not register - // side outputs but then outputs using a freshly created TupleTag. - throw new RuntimeException("sideOutput() is not available when grouping by window."); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - sideOutput(tag, output); - } - - @Override - protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { - Accumulator acc = getRuntimeContext().getAccumulator(name); - if (acc != null) { - AccumulatorHelper.compareAccumulatorTypes(name, - SerializableFnAggregatorWrapper.class, acc.getClass()); - return (Aggregator) acc; - } - - SerializableFnAggregatorWrapper accumulator = - new SerializableFnAggregatorWrapper<>(combiner); - getRuntimeContext().addAccumulator(name, accumulator); - return accumulator; - } - } - - ////////////// Checkpointing implementation //////////////// - - @Override - public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception { - StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp); - AbstractStateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp); - StateCheckpointWriter writer = StateCheckpointWriter.create(out); - Coder keyCoder = inputKvCoder.getKeyCoder(); - - // checkpoint the timers - StateCheckpointUtils.encodeTimers(activeTimers, writer, keyCoder); - - // checkpoint the state - StateCheckpointUtils.encodeState(perKeyStateInternals, writer, keyCoder); - - // checkpoint the timerInternals - context.timerInternals.encodeTimerInternals(context, writer, - inputKvCoder, windowingStrategy.getWindowFn().windowCoder()); - - taskState.setOperatorState(out.closeAndGetHandle()); - return taskState; - } - - @Override - public void restoreState(StreamTaskState taskState, long recoveryTimestamp) throws Exception { - super.restoreState(taskState, recoveryTimestamp); - - final ClassLoader userClassloader = getUserCodeClassloader(); - - Coder windowCoder = this.windowingStrategy.getWindowFn().windowCoder(); - Coder keyCoder = inputKvCoder.getKeyCoder(); - - @SuppressWarnings("unchecked") - StateHandle inputState = (StateHandle) taskState.getOperatorState(); - DataInputView in = inputState.getState(userClassloader); - StateCheckpointReader reader = new StateCheckpointReader(in); - - // restore the timers - this.activeTimers = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder); - - // restore the state - this.perKeyStateInternals = StateCheckpointUtils.decodeState( - reader, windowingStrategy.getOutputTimeFn(), keyCoder, windowCoder, userClassloader); - - // restore the timerInternals. - this.timerInternals.restoreTimerInternals(reader, inputKvCoder, windowCoder); - } + extends AbstractStreamOperator>> + implements OneInputStreamOperator>, WindowedValue>> { + + private static final long serialVersionUID = 1L; + + private transient PipelineOptions options; + + private transient CoderRegistry coderRegistry; + + private DoFn, KV> operator; + + private ProcessContext context; + + private final WindowingStrategy, BoundedWindow> windowingStrategy; + + private final Combine.KeyedCombineFn combineFn; + + private final KvCoder inputKvCoder; + + /** + * State is kept per-key. This data structure keeps this mapping between an active key, i.e. a + * key whose elements are currently waiting to be processed, and its associated state. + */ + private Map> perKeyStateInternals = new HashMap<>(); + + /** + * Timers waiting to be processed. + */ + private Map> activeTimers = new HashMap<>(); + + private FlinkTimerInternals timerInternals = new FlinkTimerInternals(); + + /** + * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy. + * This method assumes that elements are already grouped by key. + *

    + * The difference with {@link #createForIterable(PipelineOptions, PCollection, KeyedStream)} + * is that this method assumes that a combiner function is provided + * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}). + * A combiner helps at increasing the speed and, in most of the cases, reduce the per-window state. + * + * @param options the general job configuration options. + * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}. + * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key. + * @param combiner the combiner to be used. + * @param outputKvCoder the type of the output values. + */ + public static DataStream>> create( + PipelineOptions options, + PCollection input, + KeyedStream>, K> groupedStreamByKey, + Combine.KeyedCombineFn combiner, + KvCoder outputKvCoder) { + Preconditions.checkNotNull(options); + + KvCoder inputKvCoder = (KvCoder) input.getCoder(); + FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper<>(options, + input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, combiner); + + Coder>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of( + outputKvCoder, + input.getWindowingStrategy().getWindowFn().windowCoder()); + + CoderTypeInformation>> outputTypeInfo = + new CoderTypeInformation<>(windowedOutputElemCoder); + + DataStream>> groupedByKeyAndWindow = groupedStreamByKey + .transform("GroupByWindowWithCombiner", + new CoderTypeInformation<>(outputKvCoder), + windower) + .returns(outputTypeInfo); + + return groupedByKeyAndWindow; + } + + /** + * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy. + * This method assumes that elements are already grouped by key. + *

    + * The difference with {@link #create(PipelineOptions, PCollection, KeyedStream, Combine.KeyedCombineFn, KvCoder)} + * is that this method assumes no combiner function + * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}). + * + * @param options the general job configuration options. + * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}. + * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key. + */ + public static DataStream>>> createForIterable( + PipelineOptions options, + PCollection input, + KeyedStream>, K> groupedStreamByKey) { + Preconditions.checkNotNull(options); + + KvCoder inputKvCoder = (KvCoder) input.getCoder(); + Coder keyCoder = inputKvCoder.getKeyCoder(); + Coder inputValueCoder = inputKvCoder.getValueCoder(); + + FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper(options, + input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, null); + + Coder> valueIterCoder = IterableCoder.of(inputValueCoder); + KvCoder> outputElemCoder = KvCoder.of(keyCoder, valueIterCoder); + + Coder>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of( + outputElemCoder, + input.getWindowingStrategy().getWindowFn().windowCoder()); + + CoderTypeInformation>>> outputTypeInfo = + new CoderTypeInformation<>(windowedOutputElemCoder); + + DataStream>>> groupedByKeyAndWindow = groupedStreamByKey + .transform("GroupByWindow", + new CoderTypeInformation<>(windowedOutputElemCoder), + windower) + .returns(outputTypeInfo); + + return groupedByKeyAndWindow; + } + + public static FlinkGroupAlsoByWindowWrapper + createForTesting(PipelineOptions options, + CoderRegistry registry, + WindowingStrategy, BoundedWindow> windowingStrategy, + KvCoder inputCoder, + Combine.KeyedCombineFn combiner) { + Preconditions.checkNotNull(options); + + return new FlinkGroupAlsoByWindowWrapper(options, registry, windowingStrategy, inputCoder, combiner); + } + + private FlinkGroupAlsoByWindowWrapper(PipelineOptions options, + CoderRegistry registry, + WindowingStrategy, BoundedWindow> windowingStrategy, + KvCoder inputCoder, + Combine.KeyedCombineFn combiner) { + Preconditions.checkNotNull(options); + + this.options = Preconditions.checkNotNull(options); + this.coderRegistry = Preconditions.checkNotNull(registry); + this.inputKvCoder = Preconditions.checkNotNull(inputCoder);//(KvCoder) input.getCoder(); + this.windowingStrategy = Preconditions.checkNotNull(windowingStrategy);//input.getWindowingStrategy(); + this.combineFn = combiner; + this.operator = createGroupAlsoByWindowOperator(); + this.chainingStrategy = ChainingStrategy.ALWAYS; + } + + @Override + public void open() throws Exception { + super.open(); + this.context = new ProcessContext(operator, new TimestampedCollector<>(output), this.timerInternals); + } + + /** + * Create the adequate {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn}, + * if not already created. + * If a {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn} was provided, then + * a function with that combiner is created, so that elements are combined as they arrive. This is + * done for speed and (in most of the cases) for reduction of the per-window state. + */ + private DoFn, KV> createGroupAlsoByWindowOperator() { + if (this.operator == null) { + if (this.combineFn == null) { + // Thus VOUT == Iterable + Coder inputValueCoder = inputKvCoder.getValueCoder(); + + this.operator = (DoFn) GroupAlsoByWindowViaWindowSetDoFn.create( + (WindowingStrategy) this.windowingStrategy, SystemReduceFn.buffering(inputValueCoder)); + } else { + Coder inputKeyCoder = inputKvCoder.getKeyCoder(); + + AppliedCombineFn appliedCombineFn = AppliedCombineFn + .withInputCoder(combineFn, coderRegistry, inputKvCoder); + + this.operator = GroupAlsoByWindowViaWindowSetDoFn.create( + (WindowingStrategy) this.windowingStrategy, SystemReduceFn.combining(inputKeyCoder, appliedCombineFn)); + } + } + return this.operator; + } + + private void processKeyedWorkItem(KeyedWorkItem workItem) throws Exception { + context.setElement(workItem, getStateInternalsForKey(workItem.key())); + + // TODO: Ideally startBundle/finishBundle would be called when the operator is first used / about to be discarded. + operator.startBundle(context); + operator.processElement(context); + operator.finishBundle(context); + } + + @Override + public void processElement(StreamRecord>> element) throws Exception { + ArrayList> elements = new ArrayList<>(); + elements.add(WindowedValue.of(element.getValue().getValue().getValue(), element.getValue().getTimestamp(), + element.getValue().getWindows(), element.getValue().getPane())); + processKeyedWorkItem(KeyedWorkItems.elementsWorkItem(element.getValue().getValue().getKey(), elements)); + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + context.setCurrentInputWatermark(new Instant(mark.getTimestamp())); + + Multimap timers = getTimersReadyToProcess(mark.getTimestamp()); + if (!timers.isEmpty()) { + for (K key : timers.keySet()) { + processKeyedWorkItem(KeyedWorkItems.timersWorkItem(key, timers.get(key))); + } + } + + /** + * This is to take into account the different semantics of the Watermark in Flink and + * in Dataflow. To understand the reasoning behind the Dataflow semantics and its + * watermark holding logic, see the documentation of + * {@link WatermarkHold#addHold(ReduceFn.ProcessValueContext, boolean)} + * */ + long millis = Long.MAX_VALUE; + for (FlinkStateInternals state : perKeyStateInternals.values()) { + Instant watermarkHold = state.getWatermarkHold(); + if (watermarkHold != null && watermarkHold.getMillis() < millis) { + millis = watermarkHold.getMillis(); + } + } + + if (mark.getTimestamp() < millis) { + millis = mark.getTimestamp(); + } + + context.setCurrentOutputWatermark(new Instant(millis)); + + // Don't forget to re-emit the watermark for further operators down the line. + // This is critical for jobs with multiple aggregation steps. + // Imagine a job with a groupByKey() on key K1, followed by a map() that changes + // the key K1 to K2, and another groupByKey() on K2. In this case, if the watermark + // is not re-emitted, the second aggregation would never be triggered, and no result + // will be produced. + output.emitWatermark(new Watermark(millis)); + } + + @Override + public void close() throws Exception { + super.close(); + } + + private void registerActiveTimer(K key, TimerInternals.TimerData timer) { + Set timersForKey = activeTimers.get(key); + if (timersForKey == null) { + timersForKey = new HashSet<>(); + } + timersForKey.add(timer); + activeTimers.put(key, timersForKey); + } + + private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) { + Set timersForKey = activeTimers.get(key); + if (timersForKey != null) { + timersForKey.remove(timer); + if (timersForKey.isEmpty()) { + activeTimers.remove(key); + } else { + activeTimers.put(key, timersForKey); + } + } + } + + /** + * Returns the list of timers that are ready to fire. These are the timers + * that are registered to be triggered at a time before the current watermark. + * We keep these timers in a Set, so that they are deduplicated, as the same + * timer can be registered multiple times. + */ + private Multimap getTimersReadyToProcess(long currentWatermark) { + + // we keep the timers to return in a different list and launch them later + // because we cannot prevent a trigger from registering another trigger, + // which would lead to concurrent modification exception. + Multimap toFire = HashMultimap.create(); + + Iterator>> it = activeTimers.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry> keyWithTimers = it.next(); + + Iterator timerIt = keyWithTimers.getValue().iterator(); + while (timerIt.hasNext()) { + TimerInternals.TimerData timerData = timerIt.next(); + if (timerData.getTimestamp().isBefore(currentWatermark)) { + toFire.put(keyWithTimers.getKey(), timerData); + timerIt.remove(); + } + } + + if (keyWithTimers.getValue().isEmpty()) { + it.remove(); + } + } + return toFire; + } + + /** + * Gets the state associated with the specified key. + * + * @param key the key whose state we want. + * @return The {@link FlinkStateInternals} + * associated with that key. + */ + private FlinkStateInternals getStateInternalsForKey(K key) { + FlinkStateInternals stateInternals = perKeyStateInternals.get(key); + if (stateInternals == null) { + Coder windowCoder = this.windowingStrategy.getWindowFn().windowCoder(); + OutputTimeFn outputTimeFn = this.windowingStrategy.getWindowFn().getOutputTimeFn(); + stateInternals = new FlinkStateInternals<>(key, inputKvCoder.getKeyCoder(), windowCoder, outputTimeFn); + perKeyStateInternals.put(key, stateInternals); + } + return stateInternals; + } + + private class FlinkTimerInternals extends AbstractFlinkTimerInternals { + @Override + public void setTimer(TimerData timerKey) { + registerActiveTimer(context.element().key(), timerKey); + } + + @Override + public void deleteTimer(TimerData timerKey) { + unregisterActiveTimer(context.element().key(), timerKey); + } + } + + private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn>.ProcessContext { + + private final FlinkTimerInternals timerInternals; + + private final TimestampedCollector>> collector; + + private FlinkStateInternals stateInternals; + + private KeyedWorkItem element; + + public ProcessContext(DoFn, KV> function, + TimestampedCollector>> outCollector, + FlinkTimerInternals timerInternals) { + function.super(); + super.setupDelegateAggregators(); + + this.collector = Preconditions.checkNotNull(outCollector); + this.timerInternals = Preconditions.checkNotNull(timerInternals); + } + + public void setElement(KeyedWorkItem element, + FlinkStateInternals stateForKey) { + this.element = element; + this.stateInternals = stateForKey; + } + + public void setCurrentInputWatermark(Instant watermark) { + this.timerInternals.setCurrentInputWatermark(watermark); + } + + public void setCurrentOutputWatermark(Instant watermark) { + this.timerInternals.setCurrentOutputWatermark(watermark); + } + + @Override + public KeyedWorkItem element() { + return this.element; + } + + @Override + public Instant timestamp() { + throw new UnsupportedOperationException("timestamp() is not available when processing KeyedWorkItems."); + } + + @Override + public PipelineOptions getPipelineOptions() { + // TODO: PipelineOptions need to be available on the workers. + // Ideally they are captured as part of the pipeline. + // For now, construct empty options so that StateContexts.createFromComponents + // will yield a valid StateContext, which is needed to support the StateContext.window(). + if (options == null) { + options = new PipelineOptions() { + @Override + public T as(Class kls) { + return null; + } + + @Override + public T cloneAs(Class kls) { + return null; + } + + @Override + public Class> getRunner() { + return null; + } + + @Override + public void setRunner(Class> kls) { + + } + + @Override + public CheckEnabled getStableUniqueNames() { + return null; + } + + @Override + public void setStableUniqueNames(CheckEnabled enabled) { + } + }; + } + return options; + } + + @Override + public void output(KV output) { + throw new UnsupportedOperationException( + "output() is not available when processing KeyedWorkItems."); + } + + @Override + public void outputWithTimestamp(KV output, Instant timestamp) { + throw new UnsupportedOperationException( + "outputWithTimestamp() is not available when processing KeyedWorkItems."); + } + + @Override + public PaneInfo pane() { + throw new UnsupportedOperationException("pane() is not available when processing KeyedWorkItems."); + } + + @Override + public BoundedWindow window() { + throw new UnsupportedOperationException( + "window() is not available when processing KeyedWorkItems."); + } + + @Override + public WindowingInternals, KV> windowingInternals() { + return new WindowingInternals, KV>() { + + @Override + public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() { + return stateInternals; + } + + @Override + public void outputWindowedValue(KV output, Instant timestamp, Collection windows, PaneInfo pane) { + // TODO: No need to represent timestamp twice. + collector.setAbsoluteTimestamp(timestamp.getMillis()); + collector.collect(WindowedValue.of(output, timestamp, windows, pane)); + + } + + @Override + public TimerInternals timerInternals() { + return timerInternals; + } + + @Override + public Collection windows() { + throw new UnsupportedOperationException("windows() is not available in Streaming mode."); + } + + @Override + public PaneInfo pane() { + throw new UnsupportedOperationException("pane() is not available in Streaming mode."); + } + + @Override + public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { + throw new RuntimeException("writePCollectionViewData() not available in Streaming mode."); + } + + @Override + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + throw new RuntimeException("sideInput() is not available in Streaming mode."); + } + }; + } + + @Override + public T sideInput(PCollectionView view) { + throw new RuntimeException("sideInput() is not supported in Streaming mode."); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + // ignore the side output, this can happen when a user does not register + // side outputs but then outputs using a freshly created TupleTag. + throw new RuntimeException("sideOutput() is not available when grouping by window."); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + sideOutput(tag, output); + } + + @Override + protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { + Accumulator acc = getRuntimeContext().getAccumulator(name); + if (acc != null) { + AccumulatorHelper.compareAccumulatorTypes(name, + SerializableFnAggregatorWrapper.class, acc.getClass()); + return (Aggregator) acc; + } + + SerializableFnAggregatorWrapper accumulator = + new SerializableFnAggregatorWrapper<>(combiner); + getRuntimeContext().addAccumulator(name, accumulator); + return accumulator; + } + } + + ////////////// Checkpointing implementation //////////////// + + @Override + public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception { + StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp); + AbstractStateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp); + StateCheckpointWriter writer = StateCheckpointWriter.create(out); + Coder keyCoder = inputKvCoder.getKeyCoder(); + + // checkpoint the timers + StateCheckpointUtils.encodeTimers(activeTimers, writer, keyCoder); + + // checkpoint the state + StateCheckpointUtils.encodeState(perKeyStateInternals, writer, keyCoder); + + // checkpoint the timerInternals + context.timerInternals.encodeTimerInternals(context, writer, + inputKvCoder, windowingStrategy.getWindowFn().windowCoder()); + + taskState.setOperatorState(out.closeAndGetHandle()); + return taskState; + } + + @Override + public void restoreState(StreamTaskState taskState, long recoveryTimestamp) throws Exception { + super.restoreState(taskState, recoveryTimestamp); + + final ClassLoader userClassloader = getUserCodeClassloader(); + + Coder windowCoder = this.windowingStrategy.getWindowFn().windowCoder(); + Coder keyCoder = inputKvCoder.getKeyCoder(); + + @SuppressWarnings("unchecked") + StateHandle inputState = (StateHandle) taskState.getOperatorState(); + DataInputView in = inputState.getState(userClassloader); + StateCheckpointReader reader = new StateCheckpointReader(in); + + // restore the timers + this.activeTimers = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder); + + // restore the state + this.perKeyStateInternals = StateCheckpointUtils.decodeState( + reader, windowingStrategy.getOutputTimeFn(), keyCoder, windowCoder, userClassloader); + + // restore the timerInternals. + this.timerInternals.restoreTimerInternals(reader, inputKvCoder, windowCoder); + } } \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java index 24f6d40999c1..d01cf81a12c3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java @@ -34,31 +34,31 @@ * */ public class FlinkGroupByKeyWrapper { - /** - * Just an auxiliary interface to bypass the fact that java anonymous classes cannot implement - * multiple interfaces. - */ - private interface KeySelectorWithQueryableResultType extends KeySelector>, K>, ResultTypeQueryable { - } + /** + * Just an auxiliary interface to bypass the fact that java anonymous classes cannot implement + * multiple interfaces. + */ + private interface KeySelectorWithQueryableResultType extends KeySelector>, K>, ResultTypeQueryable { + } - public static KeyedStream>, K> groupStreamByKey(DataStream>> inputDataStream, KvCoder inputKvCoder) { - final Coder keyCoder = inputKvCoder.getKeyCoder(); - final TypeInformation keyTypeInfo = new CoderTypeInformation<>(keyCoder); - final boolean isKeyVoid = keyCoder instanceof VoidCoder; + public static KeyedStream>, K> groupStreamByKey(DataStream>> inputDataStream, KvCoder inputKvCoder) { + final Coder keyCoder = inputKvCoder.getKeyCoder(); + final TypeInformation keyTypeInfo = new CoderTypeInformation<>(keyCoder); + final boolean isKeyVoid = keyCoder instanceof VoidCoder; - return inputDataStream.keyBy( - new KeySelectorWithQueryableResultType() { + return inputDataStream.keyBy( + new KeySelectorWithQueryableResultType() { - @Override - public K getKey(WindowedValue> value) throws Exception { - return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE : - value.getValue().getKey(); - } + @Override + public K getKey(WindowedValue> value) throws Exception { + return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE : + value.getValue().getKey(); + } - @Override - public TypeInformation getProducedType() { - return keyTypeInfo; - } - }); - } + @Override + public TypeInformation getProducedType() { + return keyTypeInfo; + } + }); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java index d65cbc3a6041..066a55ca1fb2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java @@ -33,43 +33,43 @@ * */ public class FlinkParDoBoundMultiWrapper extends FlinkAbstractParDoWrapper { - private final TupleTag mainTag; - private final Map, Integer> outputLabels; + private final TupleTag mainTag; + private final Map, Integer> outputLabels; - public FlinkParDoBoundMultiWrapper(PipelineOptions options, WindowingStrategy windowingStrategy, DoFn doFn, TupleTag mainTag, Map, Integer> tagsToLabels) { - super(options, windowingStrategy, doFn); - this.mainTag = Preconditions.checkNotNull(mainTag); - this.outputLabels = Preconditions.checkNotNull(tagsToLabels); - } + public FlinkParDoBoundMultiWrapper(PipelineOptions options, WindowingStrategy windowingStrategy, DoFn doFn, TupleTag mainTag, Map, Integer> tagsToLabels) { + super(options, windowingStrategy, doFn); + this.mainTag = Preconditions.checkNotNull(mainTag); + this.outputLabels = Preconditions.checkNotNull(tagsToLabels); + } - @Override - public void outputWithTimestampHelper(WindowedValue inElement, OUT output, Instant timestamp, Collector> collector) { - checkTimestamp(inElement, timestamp); - Integer index = outputLabels.get(mainTag); - collector.collect(makeWindowedValue( - new RawUnionValue(index, output), - timestamp, - inElement.getWindows(), - inElement.getPane())); - } + @Override + public void outputWithTimestampHelper(WindowedValue inElement, OUT output, Instant timestamp, Collector> collector) { + checkTimestamp(inElement, timestamp); + Integer index = outputLabels.get(mainTag); + collector.collect(makeWindowedValue( + new RawUnionValue(index, output), + timestamp, + inElement.getWindows(), + inElement.getPane())); + } - @Override - public void sideOutputWithTimestampHelper(WindowedValue inElement, T output, Instant timestamp, Collector> collector, TupleTag tag) { - checkTimestamp(inElement, timestamp); - Integer index = outputLabels.get(tag); - if (index != null) { - collector.collect(makeWindowedValue( - new RawUnionValue(index, output), - timestamp, - inElement.getWindows(), - inElement.getPane())); - } - } + @Override + public void sideOutputWithTimestampHelper(WindowedValue inElement, T output, Instant timestamp, Collector> collector, TupleTag tag) { + checkTimestamp(inElement, timestamp); + Integer index = outputLabels.get(tag); + if (index != null) { + collector.collect(makeWindowedValue( + new RawUnionValue(index, output), + timestamp, + inElement.getWindows(), + inElement.getPane())); + } + } - @Override - public WindowingInternals windowingInternalsHelper(WindowedValue inElement, Collector> outCollector) { - throw new RuntimeException("FlinkParDoBoundMultiWrapper is just an internal operator serving as " + - "an intermediate transformation for the ParDo.BoundMulti translation. windowingInternals() " + - "is not available in this class."); - } + @Override + public WindowingInternals windowingInternalsHelper(WindowedValue inElement, Collector> outCollector) { + throw new RuntimeException("FlinkParDoBoundMultiWrapper is just an internal operator serving as " + + "an intermediate transformation for the ParDo.BoundMulti translation. windowingInternals() " + + "is not available in this class."); + } } \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java index b0d8a76b03c2..b3a7090e6bb5 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java @@ -35,64 +35,64 @@ * */ public class FlinkParDoBoundWrapper extends FlinkAbstractParDoWrapper { - public FlinkParDoBoundWrapper(PipelineOptions options, WindowingStrategy windowingStrategy, DoFn doFn) { - super(options, windowingStrategy, doFn); - } + public FlinkParDoBoundWrapper(PipelineOptions options, WindowingStrategy windowingStrategy, DoFn doFn) { + super(options, windowingStrategy, doFn); + } - @Override - public void outputWithTimestampHelper(WindowedValue inElement, OUT output, Instant timestamp, Collector> collector) { - checkTimestamp(inElement, timestamp); - collector.collect(makeWindowedValue( - output, - timestamp, - inElement.getWindows(), - inElement.getPane())); - } + @Override + public void outputWithTimestampHelper(WindowedValue inElement, OUT output, Instant timestamp, Collector> collector) { + checkTimestamp(inElement, timestamp); + collector.collect(makeWindowedValue( + output, + timestamp, + inElement.getWindows(), + inElement.getPane())); + } - @Override - public void sideOutputWithTimestampHelper(WindowedValue inElement, T output, Instant timestamp, Collector> outCollector, TupleTag tag) { - // ignore the side output, this can happen when a user does not register - // side outputs but then outputs using a freshly created TupleTag. - throw new RuntimeException("sideOutput() not not available in ParDo.Bound()."); - } + @Override + public void sideOutputWithTimestampHelper(WindowedValue inElement, T output, Instant timestamp, Collector> outCollector, TupleTag tag) { + // ignore the side output, this can happen when a user does not register + // side outputs but then outputs using a freshly created TupleTag. + throw new RuntimeException("sideOutput() not not available in ParDo.Bound()."); + } - @Override - public WindowingInternals windowingInternalsHelper(final WindowedValue inElement, final Collector> collector) { - return new WindowingInternals() { - @Override - public StateInternals stateInternals() { - throw new NullPointerException("StateInternals are not available for ParDo.Bound()."); - } + @Override + public WindowingInternals windowingInternalsHelper(final WindowedValue inElement, final Collector> collector) { + return new WindowingInternals() { + @Override + public StateInternals stateInternals() { + throw new NullPointerException("StateInternals are not available for ParDo.Bound()."); + } - @Override - public void outputWindowedValue(OUT output, Instant timestamp, Collection windows, PaneInfo pane) { - collector.collect(makeWindowedValue(output, timestamp, windows, pane)); - } + @Override + public void outputWindowedValue(OUT output, Instant timestamp, Collection windows, PaneInfo pane) { + collector.collect(makeWindowedValue(output, timestamp, windows, pane)); + } - @Override - public TimerInternals timerInternals() { - throw new NullPointerException("TimeInternals are not available for ParDo.Bound()."); - } + @Override + public TimerInternals timerInternals() { + throw new NullPointerException("TimeInternals are not available for ParDo.Bound()."); + } - @Override - public Collection windows() { - return inElement.getWindows(); - } + @Override + public Collection windows() { + return inElement.getWindows(); + } - @Override - public PaneInfo pane() { - return inElement.getPane(); - } + @Override + public PaneInfo pane() { + return inElement.getPane(); + } - @Override - public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { - throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode."); - } + @Override + public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { + throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode."); + } - @Override - public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { - throw new RuntimeException("sideInput() not implemented."); - } - }; - } + @Override + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + throw new RuntimeException("sideInput() not implemented."); + } + }; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java index dc8e05a39bbf..39770c924ff4 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java @@ -34,30 +34,30 @@ */ public class FlinkStreamingCreateFunction implements FlatMapFunction> { - private final List elements; - private final Coder coder; - - public FlinkStreamingCreateFunction(List elements, Coder coder) { - this.elements = elements; - this.coder = coder; - } - - @Override - public void flatMap(IN value, Collector> out) throws Exception { - - @SuppressWarnings("unchecked") - OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE; - for (byte[] element : elements) { - ByteArrayInputStream bai = new ByteArrayInputStream(element); - OUT outValue = coder.decode(bai, Coder.Context.OUTER); - - if (outValue == null) { - out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - } else { - out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - } - } - - out.close(); - } + private final List elements; + private final Coder coder; + + public FlinkStreamingCreateFunction(List elements, Coder coder) { + this.elements = elements; + this.coder = coder; + } + + @Override + public void flatMap(IN value, Collector> out) throws Exception { + + @SuppressWarnings("unchecked") + OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE; + for (byte[] element : elements) { + ByteArrayInputStream bai = new ByteArrayInputStream(element); + OUT outValue = coder.decode(bai, Coder.Context.OUTER); + + if (outValue == null) { + out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + } else { + out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + } + } + + out.close(); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java index 699d2565729f..4d6f4e245137 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java @@ -31,50 +31,50 @@ * */ public class UnboundedFlinkSource extends UnboundedSource { - private final PipelineOptions options; - private final RichParallelSourceFunction flinkSource; + private final PipelineOptions options; + private final RichParallelSourceFunction flinkSource; - public UnboundedFlinkSource(PipelineOptions pipelineOptions, RichParallelSourceFunction source) { - if(!pipelineOptions.getRunner().equals(FlinkPipelineRunner.class)) { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); - } - options = Preconditions.checkNotNull(pipelineOptions); - flinkSource = Preconditions.checkNotNull(source); - validate(); - } + public UnboundedFlinkSource(PipelineOptions pipelineOptions, RichParallelSourceFunction source) { + if(!pipelineOptions.getRunner().equals(FlinkPipelineRunner.class)) { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } + options = Preconditions.checkNotNull(pipelineOptions); + flinkSource = Preconditions.checkNotNull(source); + validate(); + } - public RichParallelSourceFunction getFlinkSource() { - return this.flinkSource; - } + public RichParallelSourceFunction getFlinkSource() { + return this.flinkSource; + } - @Override - public List> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); - } + @Override + public List> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } - @Override - public UnboundedReader createReader(PipelineOptions options, @Nullable C checkpointMark) { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); - } + @Override + public UnboundedReader createReader(PipelineOptions options, @Nullable C checkpointMark) { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } - @Nullable - @Override - public Coder getCheckpointMarkCoder() { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); - } + @Nullable + @Override + public Coder getCheckpointMarkCoder() { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } - @Override - public void validate() { - Preconditions.checkNotNull(options); - Preconditions.checkNotNull(flinkSource); - if(!options.getRunner().equals(FlinkPipelineRunner.class)) { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); - } - } + @Override + public void validate() { + Preconditions.checkNotNull(options); + Preconditions.checkNotNull(flinkSource); + if(!options.getRunner().equals(FlinkPipelineRunner.class)) { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } + } - @Override - public Coder getDefaultOutputCoder() { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); - } + @Override + public Coder getDefaultOutputCoder() { + throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner."); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java index dd14f68867fe..0b1a5daa317c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java @@ -41,191 +41,191 @@ * */ public class UnboundedSocketSource extends UnboundedSource { - private static final Coder DEFAULT_SOCKET_CODER = StringUtf8Coder.of(); + private static final Coder DEFAULT_SOCKET_CODER = StringUtf8Coder.of(); - private static final long serialVersionUID = 1L; - - private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500; - - private static final int CONNECTION_TIMEOUT_TIME = 0; - - private final String hostname; - private final int port; - private final char delimiter; - private final long maxNumRetries; - private final long delayBetweenRetries; - - public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries) { - this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP); - } - - public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) { - this.hostname = hostname; - this.port = port; - this.delimiter = delimiter; - this.maxNumRetries = maxNumRetries; - this.delayBetweenRetries = delayBetweenRetries; - } - - public String getHostname() { - return this.hostname; - } - - public int getPort() { - return this.port; - } - - public char getDelimiter() { - return this.delimiter; - } - - public long getMaxNumRetries() { - return this.maxNumRetries; - } - - public long getDelayBetweenRetries() { - return this.delayBetweenRetries; - } - - @Override - public List> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception { - return Collections.>singletonList(this); - } - - @Override - public UnboundedReader createReader(PipelineOptions options, @Nullable C checkpointMark) { - return new UnboundedSocketReader(this); - } - - @Nullable - @Override - public Coder getCheckpointMarkCoder() { - // Flink and Dataflow have different checkpointing mechanisms. - // In our case we do not need a coder. - return null; - } - - @Override - public void validate() { - checkArgument(port > 0 && port < 65536, "port is out of range"); - checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)"); - checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive"); - } - - @Override - public Coder getDefaultOutputCoder() { - return DEFAULT_SOCKET_CODER; - } - - public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader implements Serializable { - - private static final long serialVersionUID = 7526472295622776147L; - private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class); - - private final UnboundedSocketSource source; - - private Socket socket; - private BufferedReader reader; - - private boolean isRunning; - - private String currentRecord; - - public UnboundedSocketReader(UnboundedSocketSource source) { - this.source = source; - } - - private void openConnection() throws IOException { - this.socket = new Socket(); - this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME); - this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream())); - this.isRunning = true; - } - - @Override - public boolean start() throws IOException { - int attempt = 0; - while (!isRunning) { - try { - openConnection(); - LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort()); - - return advance(); - } catch (IOException e) { - LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs..."); - - if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) { - try { - Thread.sleep(this.source.getDelayBetweenRetries()); - } catch (InterruptedException e1) { - e1.printStackTrace(); - } - } else { - this.isRunning = false; - break; - } - } - } - LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort()); - return false; - } - - @Override - public boolean advance() throws IOException { - final StringBuilder buffer = new StringBuilder(); - int data; - while (isRunning && (data = reader.read()) != -1) { - // check if the string is complete - if (data != this.source.getDelimiter()) { - buffer.append((char) data); - } else { - if (buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') { - buffer.setLength(buffer.length() - 1); - } - this.currentRecord = buffer.toString(); - buffer.setLength(0); - return true; - } - } - return false; - } - - @Override - public byte[] getCurrentRecordId() throws NoSuchElementException { - return new byte[0]; - } - - @Override - public String getCurrent() throws NoSuchElementException { - return this.currentRecord; - } - - @Override - public Instant getCurrentTimestamp() throws NoSuchElementException { - return Instant.now(); - } - - @Override - public void close() throws IOException { - this.reader.close(); - this.socket.close(); - this.isRunning = false; - LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + "."); - } - - @Override - public Instant getWatermark() { - return Instant.now(); - } - - @Override - public CheckpointMark getCheckpointMark() { - return null; - } - - @Override - public UnboundedSource getCurrentSource() { - return this.source; - } - } + private static final long serialVersionUID = 1L; + + private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500; + + private static final int CONNECTION_TIMEOUT_TIME = 0; + + private final String hostname; + private final int port; + private final char delimiter; + private final long maxNumRetries; + private final long delayBetweenRetries; + + public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries) { + this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP); + } + + public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) { + this.hostname = hostname; + this.port = port; + this.delimiter = delimiter; + this.maxNumRetries = maxNumRetries; + this.delayBetweenRetries = delayBetweenRetries; + } + + public String getHostname() { + return this.hostname; + } + + public int getPort() { + return this.port; + } + + public char getDelimiter() { + return this.delimiter; + } + + public long getMaxNumRetries() { + return this.maxNumRetries; + } + + public long getDelayBetweenRetries() { + return this.delayBetweenRetries; + } + + @Override + public List> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception { + return Collections.>singletonList(this); + } + + @Override + public UnboundedReader createReader(PipelineOptions options, @Nullable C checkpointMark) { + return new UnboundedSocketReader(this); + } + + @Nullable + @Override + public Coder getCheckpointMarkCoder() { + // Flink and Dataflow have different checkpointing mechanisms. + // In our case we do not need a coder. + return null; + } + + @Override + public void validate() { + checkArgument(port > 0 && port < 65536, "port is out of range"); + checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)"); + checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive"); + } + + @Override + public Coder getDefaultOutputCoder() { + return DEFAULT_SOCKET_CODER; + } + + public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader implements Serializable { + + private static final long serialVersionUID = 7526472295622776147L; + private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class); + + private final UnboundedSocketSource source; + + private Socket socket; + private BufferedReader reader; + + private boolean isRunning; + + private String currentRecord; + + public UnboundedSocketReader(UnboundedSocketSource source) { + this.source = source; + } + + private void openConnection() throws IOException { + this.socket = new Socket(); + this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME); + this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream())); + this.isRunning = true; + } + + @Override + public boolean start() throws IOException { + int attempt = 0; + while (!isRunning) { + try { + openConnection(); + LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort()); + + return advance(); + } catch (IOException e) { + LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs..."); + + if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) { + try { + Thread.sleep(this.source.getDelayBetweenRetries()); + } catch (InterruptedException e1) { + e1.printStackTrace(); + } + } else { + this.isRunning = false; + break; + } + } + } + LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort()); + return false; + } + + @Override + public boolean advance() throws IOException { + final StringBuilder buffer = new StringBuilder(); + int data; + while (isRunning && (data = reader.read()) != -1) { + // check if the string is complete + if (data != this.source.getDelimiter()) { + buffer.append((char) data); + } else { + if (buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') { + buffer.setLength(buffer.length() - 1); + } + this.currentRecord = buffer.toString(); + buffer.setLength(0); + return true; + } + } + return false; + } + + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + return new byte[0]; + } + + @Override + public String getCurrent() throws NoSuchElementException { + return this.currentRecord; + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return Instant.now(); + } + + @Override + public void close() throws IOException { + this.reader.close(); + this.socket.close(); + this.isRunning = false; + LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + "."); + } + + @Override + public Instant getWatermark() { + return Instant.now(); + } + + @Override + public CheckpointMark getCheckpointMark() { + return null; + } + + @Override + public UnboundedSource getCurrentSource() { + return this.source; + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index e065f873673f..5a89894a7374 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -38,95 +38,95 @@ * */ public class UnboundedSourceWrapper extends RichSourceFunction> implements Triggerable { - private final String name; - private final UnboundedSource.UnboundedReader reader; - - private StreamingRuntimeContext runtime = null; - private StreamSource.ManualWatermarkContext> context = null; - - private volatile boolean isRunning = false; - - public UnboundedSourceWrapper(PipelineOptions options, Read.Unbounded transform) { - this.name = transform.getName(); - this.reader = transform.getSource().createReader(options, null); - } - - public String getName() { - return this.name; - } - - WindowedValue makeWindowedValue(T output, Instant timestamp) { - if (timestamp == null) { - timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; - } - return WindowedValue.of(output, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); - } - - @Override - public void run(SourceContext> ctx) throws Exception { - if (!(ctx instanceof StreamSource.ManualWatermarkContext)) { - throw new RuntimeException("We assume that all sources in Dataflow are EventTimeSourceFunction. " + - "Apparently " + this.name + " is not. Probably you should consider writing your own Wrapper for this source."); - } - - context = (StreamSource.ManualWatermarkContext>) ctx; - runtime = (StreamingRuntimeContext) getRuntimeContext(); - - this.isRunning = true; - boolean inputAvailable = reader.start(); - - setNextWatermarkTimer(this.runtime); - - while (isRunning) { - - while (!inputAvailable && isRunning) { - // wait a bit until we retry to pull more records - Thread.sleep(50); - inputAvailable = reader.advance(); - } - - if (inputAvailable) { - - // get it and its timestamp from the source - T item = reader.getCurrent(); - Instant timestamp = reader.getCurrentTimestamp(); - - // write it to the output collector - synchronized (ctx.getCheckpointLock()) { - context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis()); - } - - inputAvailable = reader.advance(); - } - - } - } - - @Override - public void cancel() { - isRunning = false; - } - - @Override - public void trigger(long timestamp) throws Exception { - if (this.isRunning) { - synchronized (context.getCheckpointLock()) { - long watermarkMillis = this.reader.getWatermark().getMillis(); - context.emitWatermark(new Watermark(watermarkMillis)); - } - setNextWatermarkTimer(this.runtime); - } - } - - private void setNextWatermarkTimer(StreamingRuntimeContext runtime) { - if (this.isRunning) { - long watermarkInterval = runtime.getExecutionConfig().getAutoWatermarkInterval(); - long timeToNextWatermark = getTimeToNextWaternark(watermarkInterval); - runtime.registerTimer(timeToNextWatermark, this); - } - } - - private long getTimeToNextWaternark(long watermarkInterval) { - return System.currentTimeMillis() + watermarkInterval; - } + private final String name; + private final UnboundedSource.UnboundedReader reader; + + private StreamingRuntimeContext runtime = null; + private StreamSource.ManualWatermarkContext> context = null; + + private volatile boolean isRunning = false; + + public UnboundedSourceWrapper(PipelineOptions options, Read.Unbounded transform) { + this.name = transform.getName(); + this.reader = transform.getSource().createReader(options, null); + } + + public String getName() { + return this.name; + } + + WindowedValue makeWindowedValue(T output, Instant timestamp) { + if (timestamp == null) { + timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + return WindowedValue.of(output, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); + } + + @Override + public void run(SourceContext> ctx) throws Exception { + if (!(ctx instanceof StreamSource.ManualWatermarkContext)) { + throw new RuntimeException("We assume that all sources in Dataflow are EventTimeSourceFunction. " + + "Apparently " + this.name + " is not. Probably you should consider writing your own Wrapper for this source."); + } + + context = (StreamSource.ManualWatermarkContext>) ctx; + runtime = (StreamingRuntimeContext) getRuntimeContext(); + + this.isRunning = true; + boolean inputAvailable = reader.start(); + + setNextWatermarkTimer(this.runtime); + + while (isRunning) { + + while (!inputAvailable && isRunning) { + // wait a bit until we retry to pull more records + Thread.sleep(50); + inputAvailable = reader.advance(); + } + + if (inputAvailable) { + + // get it and its timestamp from the source + T item = reader.getCurrent(); + Instant timestamp = reader.getCurrentTimestamp(); + + // write it to the output collector + synchronized (ctx.getCheckpointLock()) { + context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis()); + } + + inputAvailable = reader.advance(); + } + + } + } + + @Override + public void cancel() { + isRunning = false; + } + + @Override + public void trigger(long timestamp) throws Exception { + if (this.isRunning) { + synchronized (context.getCheckpointLock()) { + long watermarkMillis = this.reader.getWatermark().getMillis(); + context.emitWatermark(new Watermark(watermarkMillis)); + } + setNextWatermarkTimer(this.runtime); + } + } + + private void setNextWatermarkTimer(StreamingRuntimeContext runtime) { + if (this.isRunning) { + long watermarkInterval = runtime.getExecutionConfig().getAutoWatermarkInterval(); + long timeToNextWatermark = getTimeToNextWaternark(watermarkInterval); + runtime.registerTimer(timeToNextWatermark, this); + } + } + + private long getTimeToNextWaternark(long watermarkInterval) { + return System.currentTimeMillis() + watermarkInterval; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java index 84a322f68a9a..75c8ac6560c1 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java @@ -34,93 +34,93 @@ * The latter is used when snapshots of the current state are taken, for fault-tolerance. * */ public abstract class AbstractFlinkTimerInternals implements TimerInternals, Serializable { - private Instant currentInputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; - private Instant currentOutputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; - - public void setCurrentInputWatermark(Instant watermark) { - checkIfValidInputWatermark(watermark); - this.currentInputWatermark = watermark; - } - - public void setCurrentOutputWatermark(Instant watermark) { - checkIfValidOutputWatermark(watermark); - this.currentOutputWatermark = watermark; - } - - private void setCurrentInputWatermarkAfterRecovery(Instant watermark) { - if (!currentInputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) { - throw new RuntimeException("Explicitly setting the input watermark is only allowed on " + - "initialization after recovery from a node failure. Apparently this is not " + - "the case here as the watermark is already set."); - } - this.currentInputWatermark = watermark; - } - - private void setCurrentOutputWatermarkAfterRecovery(Instant watermark) { - if (!currentOutputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) { - throw new RuntimeException("Explicitly setting the output watermark is only allowed on " + - "initialization after recovery from a node failure. Apparently this is not " + - "the case here as the watermark is already set."); - } - this.currentOutputWatermark = watermark; - } - - @Override - public Instant currentProcessingTime() { - return Instant.now(); - } - - @Override - public Instant currentInputWatermarkTime() { - return currentInputWatermark; - } - - @Nullable - @Override - public Instant currentSynchronizedProcessingTime() { - // TODO - return null; - } - - @Override - public Instant currentOutputWatermarkTime() { - return currentOutputWatermark; - } - - private void checkIfValidInputWatermark(Instant newWatermark) { - if (currentInputWatermark.isAfter(newWatermark)) { - throw new IllegalArgumentException(String.format( - "Cannot set current input watermark to %s. Newer watermarks " + - "must be no earlier than the current one (%s).", - newWatermark, currentInputWatermark)); - } - } - - private void checkIfValidOutputWatermark(Instant newWatermark) { - if (currentOutputWatermark.isAfter(newWatermark)) { - throw new IllegalArgumentException(String.format( - "Cannot set current output watermark to %s. Newer watermarks " + - "must be no earlier than the current one (%s).", - newWatermark, currentOutputWatermark)); - } - } - - public void encodeTimerInternals(DoFn.ProcessContext context, - StateCheckpointWriter writer, - KvCoder kvCoder, - Coder windowCoder) throws IOException { - if (context == null) { - throw new RuntimeException("The Context has not been initialized."); - } - - writer.setTimestamp(currentInputWatermark); - writer.setTimestamp(currentOutputWatermark); - } - - public void restoreTimerInternals(StateCheckpointReader reader, - KvCoder kvCoder, - Coder windowCoder) throws IOException { - setCurrentInputWatermarkAfterRecovery(reader.getTimestamp()); - setCurrentOutputWatermarkAfterRecovery(reader.getTimestamp()); - } + private Instant currentInputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + private Instant currentOutputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + + public void setCurrentInputWatermark(Instant watermark) { + checkIfValidInputWatermark(watermark); + this.currentInputWatermark = watermark; + } + + public void setCurrentOutputWatermark(Instant watermark) { + checkIfValidOutputWatermark(watermark); + this.currentOutputWatermark = watermark; + } + + private void setCurrentInputWatermarkAfterRecovery(Instant watermark) { + if (!currentInputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) { + throw new RuntimeException("Explicitly setting the input watermark is only allowed on " + + "initialization after recovery from a node failure. Apparently this is not " + + "the case here as the watermark is already set."); + } + this.currentInputWatermark = watermark; + } + + private void setCurrentOutputWatermarkAfterRecovery(Instant watermark) { + if (!currentOutputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) { + throw new RuntimeException("Explicitly setting the output watermark is only allowed on " + + "initialization after recovery from a node failure. Apparently this is not " + + "the case here as the watermark is already set."); + } + this.currentOutputWatermark = watermark; + } + + @Override + public Instant currentProcessingTime() { + return Instant.now(); + } + + @Override + public Instant currentInputWatermarkTime() { + return currentInputWatermark; + } + + @Nullable + @Override + public Instant currentSynchronizedProcessingTime() { + // TODO + return null; + } + + @Override + public Instant currentOutputWatermarkTime() { + return currentOutputWatermark; + } + + private void checkIfValidInputWatermark(Instant newWatermark) { + if (currentInputWatermark.isAfter(newWatermark)) { + throw new IllegalArgumentException(String.format( + "Cannot set current input watermark to %s. Newer watermarks " + + "must be no earlier than the current one (%s).", + newWatermark, currentInputWatermark)); + } + } + + private void checkIfValidOutputWatermark(Instant newWatermark) { + if (currentOutputWatermark.isAfter(newWatermark)) { + throw new IllegalArgumentException(String.format( + "Cannot set current output watermark to %s. Newer watermarks " + + "must be no earlier than the current one (%s).", + newWatermark, currentOutputWatermark)); + } + } + + public void encodeTimerInternals(DoFn.ProcessContext context, + StateCheckpointWriter writer, + KvCoder kvCoder, + Coder windowCoder) throws IOException { + if (context == null) { + throw new RuntimeException("The Context has not been initialized."); + } + + writer.setTimestamp(currentInputWatermark); + writer.setTimestamp(currentOutputWatermark); + } + + public void restoreTimerInternals(StateCheckpointReader reader, + KvCoder kvCoder, + Coder windowCoder) throws IOException { + setCurrentInputWatermarkAfterRecovery(reader.getTimestamp()); + setCurrentOutputWatermarkAfterRecovery(reader.getTimestamp()); + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java index 41ab5f0bc07e..39fec1403962 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java @@ -41,673 +41,673 @@ */ public class FlinkStateInternals implements StateInternals { - private final K key; - - private final Coder keyCoder; - - private final Coder windowCoder; - - private final OutputTimeFn outputTimeFn; - - private Instant watermarkHoldAccessor; - - public FlinkStateInternals(K key, - Coder keyCoder, - Coder windowCoder, - OutputTimeFn outputTimeFn) { - this.key = key; - this.keyCoder = keyCoder; - this.windowCoder = windowCoder; - this.outputTimeFn = outputTimeFn; - } - - public Instant getWatermarkHold() { - return watermarkHoldAccessor; - } - - /** - * This is the interface state has to implement in order for it to be fault tolerant when - * executed by the FlinkPipelineRunner. - */ - private interface CheckpointableIF { - - boolean shouldPersist(); - - void persistState(StateCheckpointWriter checkpointBuilder) throws IOException; - } - - protected final StateTable inMemoryState = new StateTable() { - @Override - protected StateTag.StateBinder binderForNamespace(final StateNamespace namespace, final StateContext c) { - return new StateTag.StateBinder() { - - @Override - public ValueState bindValue(StateTag> address, Coder coder) { - return new FlinkInMemoryValue<>(encodeKey(namespace, address), coder); - } - - @Override - public BagState bindBag(StateTag> address, Coder elemCoder) { - return new FlinkInMemoryBag<>(encodeKey(namespace, address), elemCoder); - } - - @Override - public AccumulatorCombiningState bindCombiningValue( - StateTag> address, - Coder accumCoder, Combine.CombineFn combineFn) { - return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c); - } - - @Override - public AccumulatorCombiningState bindKeyedCombiningValue( - StateTag> address, - Coder accumCoder, - Combine.KeyedCombineFn combineFn) { - return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c); - } - - @Override - public AccumulatorCombiningState bindKeyedCombiningValueWithContext( - StateTag> address, - Coder accumCoder, - CombineWithContext.KeyedCombineFnWithContext combineFn) { - return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c); - } - - @Override - public WatermarkHoldState bindWatermark(StateTag> address, OutputTimeFn outputTimeFn) { - return new FlinkWatermarkHoldStateImpl<>(encodeKey(namespace, address), outputTimeFn); - } - }; - } - }; - - @Override - public K getKey() { - return key; - } - - @Override - public StateT state(StateNamespace namespace, StateTag address) { - return inMemoryState.get(namespace, address, null); - } - - @Override - public T state(StateNamespace namespace, StateTag address, StateContext c) { - return inMemoryState.get(namespace, address, c); - } - - public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { - checkpointBuilder.writeInt(getNoOfElements()); - - for (State location : inMemoryState.values()) { - if (!(location instanceof CheckpointableIF)) { - throw new IllegalStateException(String.format( - "%s wasn't created by %s -- unable to persist it", - location.getClass().getSimpleName(), - getClass().getSimpleName())); - } - ((CheckpointableIF) location).persistState(checkpointBuilder); - } - } - - public void restoreState(StateCheckpointReader checkpointReader, ClassLoader loader) - throws IOException, ClassNotFoundException { - - // the number of elements to read. - int noOfElements = checkpointReader.getInt(); - for (int i = 0; i < noOfElements; i++) { - decodeState(checkpointReader, loader); - } - } - - /** - * We remove the first character which encodes the type of the stateTag ('s' for system - * and 'u' for user). For more details check out the source of - * {@link StateTags.StateTagBase#getId()}. - */ - private void decodeState(StateCheckpointReader reader, ClassLoader loader) - throws IOException, ClassNotFoundException { - - StateType stateItemType = StateType.deserialize(reader); - ByteString stateKey = reader.getTag(); - - // first decode the namespace and the tagId... - String[] namespaceAndTag = stateKey.toStringUtf8().split("\\+"); - if (namespaceAndTag.length != 2) { - throw new IllegalArgumentException("Invalid stateKey " + stateKey.toString() + "."); - } - StateNamespace namespace = StateNamespaces.fromString(namespaceAndTag[0], windowCoder); - - // ... decide if it is a system or user stateTag... - char ownerTag = namespaceAndTag[1].charAt(0); - if (ownerTag != 's' && ownerTag != 'u') { - throw new RuntimeException("Invalid StateTag name."); - } - boolean isSystemTag = ownerTag == 's'; - String tagId = namespaceAndTag[1].substring(1); - - // ...then decode the coder (if there is one)... - Coder coder = null; - switch (stateItemType) { - case VALUE: - case LIST: - case ACCUMULATOR: - ByteString coderBytes = reader.getData(); - coder = InstantiationUtil.deserializeObject(coderBytes.toByteArray(), loader); - break; - case WATERMARK: - break; - } - - // ...then decode the combiner function (if there is one)... - CombineWithContext.KeyedCombineFnWithContext combineFn = null; - switch (stateItemType) { - case ACCUMULATOR: - ByteString combinerBytes = reader.getData(); - combineFn = InstantiationUtil.deserializeObject(combinerBytes.toByteArray(), loader); - break; - case VALUE: - case LIST: - case WATERMARK: - break; - } - - //... and finally, depending on the type of the state being decoded, - // 1) create the adequate stateTag, - // 2) create the state container, - // 3) restore the actual content. - switch (stateItemType) { - case VALUE: { - StateTag stateTag = StateTags.value(tagId, coder); - stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; - @SuppressWarnings("unchecked") - FlinkInMemoryValue value = (FlinkInMemoryValue) inMemoryState.get(namespace, stateTag, null); - value.restoreState(reader); - break; - } - case WATERMARK: { - @SuppressWarnings("unchecked") - StateTag> stateTag = StateTags.watermarkStateInternal(tagId, outputTimeFn); - stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; - @SuppressWarnings("unchecked") - FlinkWatermarkHoldStateImpl watermark = (FlinkWatermarkHoldStateImpl) inMemoryState.get(namespace, stateTag, null); - watermark.restoreState(reader); - break; - } - case LIST: { - StateTag stateTag = StateTags.bag(tagId, coder); - stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; - FlinkInMemoryBag bag = (FlinkInMemoryBag) inMemoryState.get(namespace, stateTag, null); - bag.restoreState(reader); - break; - } - case ACCUMULATOR: { - @SuppressWarnings("unchecked") - StateTag> stateTag = StateTags.keyedCombiningValueWithContext(tagId, (Coder) coder, combineFn); - stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; - @SuppressWarnings("unchecked") - FlinkInMemoryKeyedCombiningValue combiningValue = - (FlinkInMemoryKeyedCombiningValue) inMemoryState.get(namespace, stateTag, null); - combiningValue.restoreState(reader); - break; - } - default: - throw new RuntimeException("Unknown State Type " + stateItemType + "."); - } - } - - private ByteString encodeKey(StateNamespace namespace, StateTag address) { - StringBuilder sb = new StringBuilder(); - try { - namespace.appendTo(sb); - sb.append('+'); - address.appendTo(sb); - } catch (IOException e) { - throw new RuntimeException(e); - } - return ByteString.copyFromUtf8(sb.toString()); - } - - private int getNoOfElements() { - int noOfElements = 0; - for (State state : inMemoryState.values()) { - if (!(state instanceof CheckpointableIF)) { - throw new RuntimeException("State Implementations used by the " + - "Flink Dataflow Runner should implement the CheckpointableIF interface."); - } - - if (((CheckpointableIF) state).shouldPersist()) { - noOfElements++; - } - } - return noOfElements; - } - - private final class FlinkInMemoryValue implements ValueState, CheckpointableIF { - - private final ByteString stateKey; - private final Coder elemCoder; - - private T value = null; - - public FlinkInMemoryValue(ByteString stateKey, Coder elemCoder) { - this.stateKey = stateKey; - this.elemCoder = elemCoder; - } - - @Override - public void clear() { - value = null; - } - - @Override - public void write(T input) { - this.value = input; - } - - @Override - public T read() { - return value; - } - - @Override - public ValueState readLater() { - // Ignore - return this; - } - - @Override - public boolean shouldPersist() { - return value != null; - } - - @Override - public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { - if (value != null) { - // serialize the coder. - byte[] coder = InstantiationUtil.serializeObject(elemCoder); - - // encode the value into a ByteString - ByteString.Output stream = ByteString.newOutput(); - elemCoder.encode(value, stream, Coder.Context.OUTER); - ByteString data = stream.toByteString(); - - checkpointBuilder.addValueBuilder() - .setTag(stateKey) - .setData(coder) - .setData(data); - } - } - - public void restoreState(StateCheckpointReader checkpointReader) throws IOException { - ByteString valueContent = checkpointReader.getData(); - T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER); - write(outValue); - } - } - - private final class FlinkWatermarkHoldStateImpl - implements WatermarkHoldState, CheckpointableIF { - - private final ByteString stateKey; - - private Instant minimumHold = null; - - private OutputTimeFn outputTimeFn; - - public FlinkWatermarkHoldStateImpl(ByteString stateKey, OutputTimeFn outputTimeFn) { - this.stateKey = stateKey; - this.outputTimeFn = outputTimeFn; - } - - @Override - public void clear() { - // Even though we're clearing we can't remove this from the in-memory state map, since - // other users may already have a handle on this WatermarkBagInternal. - minimumHold = null; - watermarkHoldAccessor = null; - } - - @Override - public void add(Instant watermarkHold) { - if (minimumHold == null || minimumHold.isAfter(watermarkHold)) { - watermarkHoldAccessor = watermarkHold; - minimumHold = watermarkHold; - } - } - - @Override - public ReadableState isEmpty() { - return new ReadableState() { - @Override - public Boolean read() { - return minimumHold == null; - } - - @Override - public ReadableState readLater() { - // Ignore - return this; - } - }; - } - - @Override - public OutputTimeFn getOutputTimeFn() { - return outputTimeFn; - } - - @Override - public Instant read() { - return minimumHold; - } - - @Override - public WatermarkHoldState readLater() { - // Ignore - return this; - } - - @Override - public String toString() { - return Objects.toString(minimumHold); - } - - @Override - public boolean shouldPersist() { - return minimumHold != null; - } - - @Override - public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { - if (minimumHold != null) { - checkpointBuilder.addWatermarkHoldsBuilder() - .setTag(stateKey) - .setTimestamp(minimumHold); - } - } - - public void restoreState(StateCheckpointReader checkpointReader) throws IOException { - Instant watermark = checkpointReader.getTimestamp(); - add(watermark); - } - } - - - private static CombineWithContext.KeyedCombineFnWithContext withContext( - final Combine.KeyedCombineFn combineFn) { - return new CombineWithContext.KeyedCombineFnWithContext() { - @Override - public AccumT createAccumulator(K key, CombineWithContext.Context c) { - return combineFn.createAccumulator(key); - } - - @Override - public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) { - return combineFn.addInput(key, accumulator, value); - } - - @Override - public AccumT mergeAccumulators(K key, Iterable accumulators, CombineWithContext.Context c) { - return combineFn.mergeAccumulators(key, accumulators); - } - - @Override - public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) { - return combineFn.extractOutput(key, accumulator); - } - }; - } - - private static CombineWithContext.KeyedCombineFnWithContext withKeyAndContext( - final Combine.CombineFn combineFn) { - return new CombineWithContext.KeyedCombineFnWithContext() { - @Override - public AccumT createAccumulator(K key, CombineWithContext.Context c) { - return combineFn.createAccumulator(); - } - - @Override - public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) { - return combineFn.addInput(accumulator, value); - } - - @Override - public AccumT mergeAccumulators(K key, Iterable accumulators, CombineWithContext.Context c) { - return combineFn.mergeAccumulators(accumulators); - } - - @Override - public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) { - return combineFn.extractOutput(accumulator); - } - }; - } - - private final class FlinkInMemoryKeyedCombiningValue - implements AccumulatorCombiningState, CheckpointableIF { - - private final ByteString stateKey; - private final CombineWithContext.KeyedCombineFnWithContext combineFn; - private final Coder accumCoder; - private final CombineWithContext.Context context; - - private AccumT accum = null; - private boolean isClear = true; - - private FlinkInMemoryKeyedCombiningValue(ByteString stateKey, - Combine.CombineFn combineFn, - Coder accumCoder, - final StateContext stateContext) { - this(stateKey, withKeyAndContext(combineFn), accumCoder, stateContext); - } - - - private FlinkInMemoryKeyedCombiningValue(ByteString stateKey, - Combine.KeyedCombineFn combineFn, - Coder accumCoder, - final StateContext stateContext) { - this(stateKey, withContext(combineFn), accumCoder, stateContext); - } - - private FlinkInMemoryKeyedCombiningValue(ByteString stateKey, - CombineWithContext.KeyedCombineFnWithContext combineFn, - Coder accumCoder, - final StateContext stateContext) { - Preconditions.checkNotNull(combineFn); - Preconditions.checkNotNull(accumCoder); - - this.stateKey = stateKey; - this.combineFn = combineFn; - this.accumCoder = accumCoder; - this.context = new CombineWithContext.Context() { - @Override - public PipelineOptions getPipelineOptions() { - return stateContext.getPipelineOptions(); - } - - @Override - public T sideInput(PCollectionView view) { - return stateContext.sideInput(view); - } - }; - accum = combineFn.createAccumulator(key, context); - } - - @Override - public void clear() { - accum = combineFn.createAccumulator(key, context); - isClear = true; - } - - @Override - public void add(InputT input) { - isClear = false; - accum = combineFn.addInput(key, accum, input, context); - } - - @Override - public AccumT getAccum() { - return accum; - } - - @Override - public ReadableState isEmpty() { - return new ReadableState() { - @Override - public ReadableState readLater() { - // Ignore - return this; - } - - @Override - public Boolean read() { - return isClear; - } - }; - } - - @Override - public void addAccum(AccumT accum) { - isClear = false; - this.accum = combineFn.mergeAccumulators(key, Arrays.asList(this.accum, accum), context); - } - - @Override - public AccumT mergeAccumulators(Iterable accumulators) { - return combineFn.mergeAccumulators(key, accumulators, context); - } - - @Override - public OutputT read() { - return combineFn.extractOutput(key, accum, context); - } - - @Override - public AccumulatorCombiningState readLater() { - // Ignore - return this; - } - - @Override - public boolean shouldPersist() { - return !isClear; - } - - @Override - public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { - if (!isClear) { - // serialize the coder. - byte[] coder = InstantiationUtil.serializeObject(accumCoder); - - // serialize the combiner. - byte[] combiner = InstantiationUtil.serializeObject(combineFn); - - // encode the accumulator into a ByteString - ByteString.Output stream = ByteString.newOutput(); - accumCoder.encode(accum, stream, Coder.Context.OUTER); - ByteString data = stream.toByteString(); - - // put the flag that the next serialized element is an accumulator - checkpointBuilder.addAccumulatorBuilder() - .setTag(stateKey) - .setData(coder) - .setData(combiner) - .setData(data); - } - } - - public void restoreState(StateCheckpointReader checkpointReader) throws IOException { - ByteString valueContent = checkpointReader.getData(); - AccumT accum = this.accumCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER); - addAccum(accum); - } - } - - private static final class FlinkInMemoryBag implements BagState, CheckpointableIF { - private final List contents = new ArrayList<>(); - - private final ByteString stateKey; - private final Coder elemCoder; - - public FlinkInMemoryBag(ByteString stateKey, Coder elemCoder) { - this.stateKey = stateKey; - this.elemCoder = elemCoder; - } - - @Override - public void clear() { - contents.clear(); - } - - @Override - public Iterable read() { - return contents; - } - - @Override - public BagState readLater() { - // Ignore - return this; - } - - @Override - public void add(T input) { - contents.add(input); - } - - @Override - public ReadableState isEmpty() { - return new ReadableState() { - @Override - public ReadableState readLater() { - // Ignore - return this; - } - - @Override - public Boolean read() { - return contents.isEmpty(); - } - }; - } - - @Override - public boolean shouldPersist() { - return !contents.isEmpty(); - } - - @Override - public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { - if (!contents.isEmpty()) { - // serialize the coder. - byte[] coder = InstantiationUtil.serializeObject(elemCoder); - - checkpointBuilder.addListUpdatesBuilder() - .setTag(stateKey) - .setData(coder) - .writeInt(contents.size()); - - for (T item : contents) { - // encode the element - ByteString.Output stream = ByteString.newOutput(); - elemCoder.encode(item, stream, Coder.Context.OUTER); - ByteString data = stream.toByteString(); - - // add the data to the checkpoint. - checkpointBuilder.setData(data); - } - } - } - - public void restoreState(StateCheckpointReader checkpointReader) throws IOException { - int noOfValues = checkpointReader.getInt(); - for (int j = 0; j < noOfValues; j++) { - ByteString valueContent = checkpointReader.getData(); - T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER); - add(outValue); - } - } - } + private final K key; + + private final Coder keyCoder; + + private final Coder windowCoder; + + private final OutputTimeFn outputTimeFn; + + private Instant watermarkHoldAccessor; + + public FlinkStateInternals(K key, + Coder keyCoder, + Coder windowCoder, + OutputTimeFn outputTimeFn) { + this.key = key; + this.keyCoder = keyCoder; + this.windowCoder = windowCoder; + this.outputTimeFn = outputTimeFn; + } + + public Instant getWatermarkHold() { + return watermarkHoldAccessor; + } + + /** + * This is the interface state has to implement in order for it to be fault tolerant when + * executed by the FlinkPipelineRunner. + */ + private interface CheckpointableIF { + + boolean shouldPersist(); + + void persistState(StateCheckpointWriter checkpointBuilder) throws IOException; + } + + protected final StateTable inMemoryState = new StateTable() { + @Override + protected StateTag.StateBinder binderForNamespace(final StateNamespace namespace, final StateContext c) { + return new StateTag.StateBinder() { + + @Override + public ValueState bindValue(StateTag> address, Coder coder) { + return new FlinkInMemoryValue<>(encodeKey(namespace, address), coder); + } + + @Override + public BagState bindBag(StateTag> address, Coder elemCoder) { + return new FlinkInMemoryBag<>(encodeKey(namespace, address), elemCoder); + } + + @Override + public AccumulatorCombiningState bindCombiningValue( + StateTag> address, + Coder accumCoder, Combine.CombineFn combineFn) { + return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c); + } + + @Override + public AccumulatorCombiningState bindKeyedCombiningValue( + StateTag> address, + Coder accumCoder, + Combine.KeyedCombineFn combineFn) { + return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c); + } + + @Override + public AccumulatorCombiningState bindKeyedCombiningValueWithContext( + StateTag> address, + Coder accumCoder, + CombineWithContext.KeyedCombineFnWithContext combineFn) { + return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c); + } + + @Override + public WatermarkHoldState bindWatermark(StateTag> address, OutputTimeFn outputTimeFn) { + return new FlinkWatermarkHoldStateImpl<>(encodeKey(namespace, address), outputTimeFn); + } + }; + } + }; + + @Override + public K getKey() { + return key; + } + + @Override + public StateT state(StateNamespace namespace, StateTag address) { + return inMemoryState.get(namespace, address, null); + } + + @Override + public T state(StateNamespace namespace, StateTag address, StateContext c) { + return inMemoryState.get(namespace, address, c); + } + + public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { + checkpointBuilder.writeInt(getNoOfElements()); + + for (State location : inMemoryState.values()) { + if (!(location instanceof CheckpointableIF)) { + throw new IllegalStateException(String.format( + "%s wasn't created by %s -- unable to persist it", + location.getClass().getSimpleName(), + getClass().getSimpleName())); + } + ((CheckpointableIF) location).persistState(checkpointBuilder); + } + } + + public void restoreState(StateCheckpointReader checkpointReader, ClassLoader loader) + throws IOException, ClassNotFoundException { + + // the number of elements to read. + int noOfElements = checkpointReader.getInt(); + for (int i = 0; i < noOfElements; i++) { + decodeState(checkpointReader, loader); + } + } + + /** + * We remove the first character which encodes the type of the stateTag ('s' for system + * and 'u' for user). For more details check out the source of + * {@link StateTags.StateTagBase#getId()}. + */ + private void decodeState(StateCheckpointReader reader, ClassLoader loader) + throws IOException, ClassNotFoundException { + + StateType stateItemType = StateType.deserialize(reader); + ByteString stateKey = reader.getTag(); + + // first decode the namespace and the tagId... + String[] namespaceAndTag = stateKey.toStringUtf8().split("\\+"); + if (namespaceAndTag.length != 2) { + throw new IllegalArgumentException("Invalid stateKey " + stateKey.toString() + "."); + } + StateNamespace namespace = StateNamespaces.fromString(namespaceAndTag[0], windowCoder); + + // ... decide if it is a system or user stateTag... + char ownerTag = namespaceAndTag[1].charAt(0); + if (ownerTag != 's' && ownerTag != 'u') { + throw new RuntimeException("Invalid StateTag name."); + } + boolean isSystemTag = ownerTag == 's'; + String tagId = namespaceAndTag[1].substring(1); + + // ...then decode the coder (if there is one)... + Coder coder = null; + switch (stateItemType) { + case VALUE: + case LIST: + case ACCUMULATOR: + ByteString coderBytes = reader.getData(); + coder = InstantiationUtil.deserializeObject(coderBytes.toByteArray(), loader); + break; + case WATERMARK: + break; + } + + // ...then decode the combiner function (if there is one)... + CombineWithContext.KeyedCombineFnWithContext combineFn = null; + switch (stateItemType) { + case ACCUMULATOR: + ByteString combinerBytes = reader.getData(); + combineFn = InstantiationUtil.deserializeObject(combinerBytes.toByteArray(), loader); + break; + case VALUE: + case LIST: + case WATERMARK: + break; + } + + //... and finally, depending on the type of the state being decoded, + // 1) create the adequate stateTag, + // 2) create the state container, + // 3) restore the actual content. + switch (stateItemType) { + case VALUE: { + StateTag stateTag = StateTags.value(tagId, coder); + stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; + @SuppressWarnings("unchecked") + FlinkInMemoryValue value = (FlinkInMemoryValue) inMemoryState.get(namespace, stateTag, null); + value.restoreState(reader); + break; + } + case WATERMARK: { + @SuppressWarnings("unchecked") + StateTag> stateTag = StateTags.watermarkStateInternal(tagId, outputTimeFn); + stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; + @SuppressWarnings("unchecked") + FlinkWatermarkHoldStateImpl watermark = (FlinkWatermarkHoldStateImpl) inMemoryState.get(namespace, stateTag, null); + watermark.restoreState(reader); + break; + } + case LIST: { + StateTag stateTag = StateTags.bag(tagId, coder); + stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; + FlinkInMemoryBag bag = (FlinkInMemoryBag) inMemoryState.get(namespace, stateTag, null); + bag.restoreState(reader); + break; + } + case ACCUMULATOR: { + @SuppressWarnings("unchecked") + StateTag> stateTag = StateTags.keyedCombiningValueWithContext(tagId, (Coder) coder, combineFn); + stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag; + @SuppressWarnings("unchecked") + FlinkInMemoryKeyedCombiningValue combiningValue = + (FlinkInMemoryKeyedCombiningValue) inMemoryState.get(namespace, stateTag, null); + combiningValue.restoreState(reader); + break; + } + default: + throw new RuntimeException("Unknown State Type " + stateItemType + "."); + } + } + + private ByteString encodeKey(StateNamespace namespace, StateTag address) { + StringBuilder sb = new StringBuilder(); + try { + namespace.appendTo(sb); + sb.append('+'); + address.appendTo(sb); + } catch (IOException e) { + throw new RuntimeException(e); + } + return ByteString.copyFromUtf8(sb.toString()); + } + + private int getNoOfElements() { + int noOfElements = 0; + for (State state : inMemoryState.values()) { + if (!(state instanceof CheckpointableIF)) { + throw new RuntimeException("State Implementations used by the " + + "Flink Dataflow Runner should implement the CheckpointableIF interface."); + } + + if (((CheckpointableIF) state).shouldPersist()) { + noOfElements++; + } + } + return noOfElements; + } + + private final class FlinkInMemoryValue implements ValueState, CheckpointableIF { + + private final ByteString stateKey; + private final Coder elemCoder; + + private T value = null; + + public FlinkInMemoryValue(ByteString stateKey, Coder elemCoder) { + this.stateKey = stateKey; + this.elemCoder = elemCoder; + } + + @Override + public void clear() { + value = null; + } + + @Override + public void write(T input) { + this.value = input; + } + + @Override + public T read() { + return value; + } + + @Override + public ValueState readLater() { + // Ignore + return this; + } + + @Override + public boolean shouldPersist() { + return value != null; + } + + @Override + public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { + if (value != null) { + // serialize the coder. + byte[] coder = InstantiationUtil.serializeObject(elemCoder); + + // encode the value into a ByteString + ByteString.Output stream = ByteString.newOutput(); + elemCoder.encode(value, stream, Coder.Context.OUTER); + ByteString data = stream.toByteString(); + + checkpointBuilder.addValueBuilder() + .setTag(stateKey) + .setData(coder) + .setData(data); + } + } + + public void restoreState(StateCheckpointReader checkpointReader) throws IOException { + ByteString valueContent = checkpointReader.getData(); + T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER); + write(outValue); + } + } + + private final class FlinkWatermarkHoldStateImpl + implements WatermarkHoldState, CheckpointableIF { + + private final ByteString stateKey; + + private Instant minimumHold = null; + + private OutputTimeFn outputTimeFn; + + public FlinkWatermarkHoldStateImpl(ByteString stateKey, OutputTimeFn outputTimeFn) { + this.stateKey = stateKey; + this.outputTimeFn = outputTimeFn; + } + + @Override + public void clear() { + // Even though we're clearing we can't remove this from the in-memory state map, since + // other users may already have a handle on this WatermarkBagInternal. + minimumHold = null; + watermarkHoldAccessor = null; + } + + @Override + public void add(Instant watermarkHold) { + if (minimumHold == null || minimumHold.isAfter(watermarkHold)) { + watermarkHoldAccessor = watermarkHold; + minimumHold = watermarkHold; + } + } + + @Override + public ReadableState isEmpty() { + return new ReadableState() { + @Override + public Boolean read() { + return minimumHold == null; + } + + @Override + public ReadableState readLater() { + // Ignore + return this; + } + }; + } + + @Override + public OutputTimeFn getOutputTimeFn() { + return outputTimeFn; + } + + @Override + public Instant read() { + return minimumHold; + } + + @Override + public WatermarkHoldState readLater() { + // Ignore + return this; + } + + @Override + public String toString() { + return Objects.toString(minimumHold); + } + + @Override + public boolean shouldPersist() { + return minimumHold != null; + } + + @Override + public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { + if (minimumHold != null) { + checkpointBuilder.addWatermarkHoldsBuilder() + .setTag(stateKey) + .setTimestamp(minimumHold); + } + } + + public void restoreState(StateCheckpointReader checkpointReader) throws IOException { + Instant watermark = checkpointReader.getTimestamp(); + add(watermark); + } + } + + + private static CombineWithContext.KeyedCombineFnWithContext withContext( + final Combine.KeyedCombineFn combineFn) { + return new CombineWithContext.KeyedCombineFnWithContext() { + @Override + public AccumT createAccumulator(K key, CombineWithContext.Context c) { + return combineFn.createAccumulator(key); + } + + @Override + public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) { + return combineFn.addInput(key, accumulator, value); + } + + @Override + public AccumT mergeAccumulators(K key, Iterable accumulators, CombineWithContext.Context c) { + return combineFn.mergeAccumulators(key, accumulators); + } + + @Override + public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) { + return combineFn.extractOutput(key, accumulator); + } + }; + } + + private static CombineWithContext.KeyedCombineFnWithContext withKeyAndContext( + final Combine.CombineFn combineFn) { + return new CombineWithContext.KeyedCombineFnWithContext() { + @Override + public AccumT createAccumulator(K key, CombineWithContext.Context c) { + return combineFn.createAccumulator(); + } + + @Override + public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) { + return combineFn.addInput(accumulator, value); + } + + @Override + public AccumT mergeAccumulators(K key, Iterable accumulators, CombineWithContext.Context c) { + return combineFn.mergeAccumulators(accumulators); + } + + @Override + public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) { + return combineFn.extractOutput(accumulator); + } + }; + } + + private final class FlinkInMemoryKeyedCombiningValue + implements AccumulatorCombiningState, CheckpointableIF { + + private final ByteString stateKey; + private final CombineWithContext.KeyedCombineFnWithContext combineFn; + private final Coder accumCoder; + private final CombineWithContext.Context context; + + private AccumT accum = null; + private boolean isClear = true; + + private FlinkInMemoryKeyedCombiningValue(ByteString stateKey, + Combine.CombineFn combineFn, + Coder accumCoder, + final StateContext stateContext) { + this(stateKey, withKeyAndContext(combineFn), accumCoder, stateContext); + } + + + private FlinkInMemoryKeyedCombiningValue(ByteString stateKey, + Combine.KeyedCombineFn combineFn, + Coder accumCoder, + final StateContext stateContext) { + this(stateKey, withContext(combineFn), accumCoder, stateContext); + } + + private FlinkInMemoryKeyedCombiningValue(ByteString stateKey, + CombineWithContext.KeyedCombineFnWithContext combineFn, + Coder accumCoder, + final StateContext stateContext) { + Preconditions.checkNotNull(combineFn); + Preconditions.checkNotNull(accumCoder); + + this.stateKey = stateKey; + this.combineFn = combineFn; + this.accumCoder = accumCoder; + this.context = new CombineWithContext.Context() { + @Override + public PipelineOptions getPipelineOptions() { + return stateContext.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + return stateContext.sideInput(view); + } + }; + accum = combineFn.createAccumulator(key, context); + } + + @Override + public void clear() { + accum = combineFn.createAccumulator(key, context); + isClear = true; + } + + @Override + public void add(InputT input) { + isClear = false; + accum = combineFn.addInput(key, accum, input, context); + } + + @Override + public AccumT getAccum() { + return accum; + } + + @Override + public ReadableState isEmpty() { + return new ReadableState() { + @Override + public ReadableState readLater() { + // Ignore + return this; + } + + @Override + public Boolean read() { + return isClear; + } + }; + } + + @Override + public void addAccum(AccumT accum) { + isClear = false; + this.accum = combineFn.mergeAccumulators(key, Arrays.asList(this.accum, accum), context); + } + + @Override + public AccumT mergeAccumulators(Iterable accumulators) { + return combineFn.mergeAccumulators(key, accumulators, context); + } + + @Override + public OutputT read() { + return combineFn.extractOutput(key, accum, context); + } + + @Override + public AccumulatorCombiningState readLater() { + // Ignore + return this; + } + + @Override + public boolean shouldPersist() { + return !isClear; + } + + @Override + public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { + if (!isClear) { + // serialize the coder. + byte[] coder = InstantiationUtil.serializeObject(accumCoder); + + // serialize the combiner. + byte[] combiner = InstantiationUtil.serializeObject(combineFn); + + // encode the accumulator into a ByteString + ByteString.Output stream = ByteString.newOutput(); + accumCoder.encode(accum, stream, Coder.Context.OUTER); + ByteString data = stream.toByteString(); + + // put the flag that the next serialized element is an accumulator + checkpointBuilder.addAccumulatorBuilder() + .setTag(stateKey) + .setData(coder) + .setData(combiner) + .setData(data); + } + } + + public void restoreState(StateCheckpointReader checkpointReader) throws IOException { + ByteString valueContent = checkpointReader.getData(); + AccumT accum = this.accumCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER); + addAccum(accum); + } + } + + private static final class FlinkInMemoryBag implements BagState, CheckpointableIF { + private final List contents = new ArrayList<>(); + + private final ByteString stateKey; + private final Coder elemCoder; + + public FlinkInMemoryBag(ByteString stateKey, Coder elemCoder) { + this.stateKey = stateKey; + this.elemCoder = elemCoder; + } + + @Override + public void clear() { + contents.clear(); + } + + @Override + public Iterable read() { + return contents; + } + + @Override + public BagState readLater() { + // Ignore + return this; + } + + @Override + public void add(T input) { + contents.add(input); + } + + @Override + public ReadableState isEmpty() { + return new ReadableState() { + @Override + public ReadableState readLater() { + // Ignore + return this; + } + + @Override + public Boolean read() { + return contents.isEmpty(); + } + }; + } + + @Override + public boolean shouldPersist() { + return !contents.isEmpty(); + } + + @Override + public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException { + if (!contents.isEmpty()) { + // serialize the coder. + byte[] coder = InstantiationUtil.serializeObject(elemCoder); + + checkpointBuilder.addListUpdatesBuilder() + .setTag(stateKey) + .setData(coder) + .writeInt(contents.size()); + + for (T item : contents) { + // encode the element + ByteString.Output stream = ByteString.newOutput(); + elemCoder.encode(item, stream, Coder.Context.OUTER); + ByteString data = stream.toByteString(); + + // add the data to the checkpoint. + checkpointBuilder.setData(data); + } + } + } + + public void restoreState(StateCheckpointReader checkpointReader) throws IOException { + int noOfValues = checkpointReader.getInt(); + for (int j = 0; j < noOfValues; j++) { + ByteString valueContent = checkpointReader.getData(); + T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER); + add(outValue); + } + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java index ba8ef8962f6c..753309ef244c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java @@ -25,65 +25,65 @@ public class StateCheckpointReader { - private final DataInputView input; - - public StateCheckpointReader(DataInputView in) { - this.input = in; - } - - public ByteString getTag() throws IOException { - return ByteString.copyFrom(readRawData()); - } - - public String getTagToString() throws IOException { - return input.readUTF(); - } - - public ByteString getData() throws IOException { - return ByteString.copyFrom(readRawData()); - } - - public int getInt() throws IOException { - validate(); - return input.readInt(); - } - - public byte getByte() throws IOException { - validate(); - return input.readByte(); - } - - public Instant getTimestamp() throws IOException { - validate(); - Long watermarkMillis = input.readLong(); - return new Instant(TimeUnit.MICROSECONDS.toMillis(watermarkMillis)); - } - - public K deserializeKey(CoderTypeSerializer keySerializer) throws IOException { - return deserializeObject(keySerializer); - } - - public T deserializeObject(CoderTypeSerializer objectSerializer) throws IOException { - return objectSerializer.deserialize(input); - } - - ///////// Helper Methods /////// - - private byte[] readRawData() throws IOException { - validate(); - int size = input.readInt(); - - byte[] serData = new byte[size]; - int bytesRead = input.read(serData); - if (bytesRead != size) { - throw new RuntimeException("Error while deserializing checkpoint. Not enough bytes in the input stream."); - } - return serData; - } - - private void validate() { - if (this.input == null) { - throw new RuntimeException("StateBackend not initialized yet."); - } - } + private final DataInputView input; + + public StateCheckpointReader(DataInputView in) { + this.input = in; + } + + public ByteString getTag() throws IOException { + return ByteString.copyFrom(readRawData()); + } + + public String getTagToString() throws IOException { + return input.readUTF(); + } + + public ByteString getData() throws IOException { + return ByteString.copyFrom(readRawData()); + } + + public int getInt() throws IOException { + validate(); + return input.readInt(); + } + + public byte getByte() throws IOException { + validate(); + return input.readByte(); + } + + public Instant getTimestamp() throws IOException { + validate(); + Long watermarkMillis = input.readLong(); + return new Instant(TimeUnit.MICROSECONDS.toMillis(watermarkMillis)); + } + + public K deserializeKey(CoderTypeSerializer keySerializer) throws IOException { + return deserializeObject(keySerializer); + } + + public T deserializeObject(CoderTypeSerializer objectSerializer) throws IOException { + return objectSerializer.deserialize(input); + } + + ///////// Helper Methods /////// + + private byte[] readRawData() throws IOException { + validate(); + int size = input.readInt(); + + byte[] serData = new byte[size]; + int bytesRead = input.read(serData); + if (bytesRead != size) { + throw new RuntimeException("Error while deserializing checkpoint. Not enough bytes in the input stream."); + } + return serData; + } + + private void validate() { + if (this.input == null) { + throw new RuntimeException("StateBackend not initialized yet."); + } + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java index cd851637bd1f..17418293ecfe 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java @@ -34,120 +34,120 @@ public class StateCheckpointUtils { - public static void encodeState(Map> perKeyStateInternals, - StateCheckpointWriter writer, Coder keyCoder) throws IOException { - CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); - - int noOfKeys = perKeyStateInternals.size(); - writer.writeInt(noOfKeys); - for (Map.Entry> keyStatePair : perKeyStateInternals.entrySet()) { - K key = keyStatePair.getKey(); - FlinkStateInternals state = keyStatePair.getValue(); - - // encode the key - writer.serializeKey(key, keySerializer); - - // write the associated state - state.persistState(writer); - } - } - - public static Map> decodeState( - StateCheckpointReader reader, - OutputTimeFn outputTimeFn, - Coder keyCoder, - Coder windowCoder, - ClassLoader classLoader) throws IOException, ClassNotFoundException { - - int noOfKeys = reader.getInt(); - Map> perKeyStateInternals = new HashMap<>(noOfKeys); - perKeyStateInternals.clear(); - - CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); - for (int i = 0; i < noOfKeys; i++) { - - // decode the key. - K key = reader.deserializeKey(keySerializer); - - //decode the state associated to the key. - FlinkStateInternals stateForKey = - new FlinkStateInternals<>(key, keyCoder, windowCoder, outputTimeFn); - stateForKey.restoreState(reader, classLoader); - perKeyStateInternals.put(key, stateForKey); - } - return perKeyStateInternals; - } - - ////////////// Encoding/Decoding the Timers //////////////// - - - public static void encodeTimers(Map> allTimers, - StateCheckpointWriter writer, - Coder keyCoder) throws IOException { - CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); - - int noOfKeys = allTimers.size(); - writer.writeInt(noOfKeys); - for (Map.Entry> timersPerKey : allTimers.entrySet()) { - K key = timersPerKey.getKey(); - - // encode the key - writer.serializeKey(key, keySerializer); - - // write the associated timers - Set timers = timersPerKey.getValue(); - encodeTimerDataForKey(writer, timers); - } - } - - public static Map> decodeTimers( - StateCheckpointReader reader, - Coder windowCoder, - Coder keyCoder) throws IOException { - - int noOfKeys = reader.getInt(); - Map> activeTimers = new HashMap<>(noOfKeys); - activeTimers.clear(); - - CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); - for (int i = 0; i < noOfKeys; i++) { - - // decode the key. - K key = reader.deserializeKey(keySerializer); - - // decode the associated timers. - Set timers = decodeTimerDataForKey(reader, windowCoder); - activeTimers.put(key, timers); - } - return activeTimers; - } - - private static void encodeTimerDataForKey(StateCheckpointWriter writer, Set timers) throws IOException { - // encode timers - writer.writeInt(timers.size()); - for (TimerInternals.TimerData timer : timers) { - String stringKey = timer.getNamespace().stringKey(); - - writer.setTag(stringKey); - writer.setTimestamp(timer.getTimestamp()); - writer.writeInt(timer.getDomain().ordinal()); - } - } - - private static Set decodeTimerDataForKey( - StateCheckpointReader reader, Coder windowCoder) throws IOException { - - // decode the timers: first their number and then the content itself. - int noOfTimers = reader.getInt(); - Set timers = new HashSet<>(noOfTimers); - for (int i = 0; i < noOfTimers; i++) { - String stringKey = reader.getTagToString(); - Instant instant = reader.getTimestamp(); - TimeDomain domain = TimeDomain.values()[reader.getInt()]; - - StateNamespace namespace = StateNamespaces.fromString(stringKey, windowCoder); - timers.add(TimerInternals.TimerData.of(namespace, instant, domain)); - } - return timers; - } + public static void encodeState(Map> perKeyStateInternals, + StateCheckpointWriter writer, Coder keyCoder) throws IOException { + CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); + + int noOfKeys = perKeyStateInternals.size(); + writer.writeInt(noOfKeys); + for (Map.Entry> keyStatePair : perKeyStateInternals.entrySet()) { + K key = keyStatePair.getKey(); + FlinkStateInternals state = keyStatePair.getValue(); + + // encode the key + writer.serializeKey(key, keySerializer); + + // write the associated state + state.persistState(writer); + } + } + + public static Map> decodeState( + StateCheckpointReader reader, + OutputTimeFn outputTimeFn, + Coder keyCoder, + Coder windowCoder, + ClassLoader classLoader) throws IOException, ClassNotFoundException { + + int noOfKeys = reader.getInt(); + Map> perKeyStateInternals = new HashMap<>(noOfKeys); + perKeyStateInternals.clear(); + + CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); + for (int i = 0; i < noOfKeys; i++) { + + // decode the key. + K key = reader.deserializeKey(keySerializer); + + //decode the state associated to the key. + FlinkStateInternals stateForKey = + new FlinkStateInternals<>(key, keyCoder, windowCoder, outputTimeFn); + stateForKey.restoreState(reader, classLoader); + perKeyStateInternals.put(key, stateForKey); + } + return perKeyStateInternals; + } + + ////////////// Encoding/Decoding the Timers //////////////// + + + public static void encodeTimers(Map> allTimers, + StateCheckpointWriter writer, + Coder keyCoder) throws IOException { + CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); + + int noOfKeys = allTimers.size(); + writer.writeInt(noOfKeys); + for (Map.Entry> timersPerKey : allTimers.entrySet()) { + K key = timersPerKey.getKey(); + + // encode the key + writer.serializeKey(key, keySerializer); + + // write the associated timers + Set timers = timersPerKey.getValue(); + encodeTimerDataForKey(writer, timers); + } + } + + public static Map> decodeTimers( + StateCheckpointReader reader, + Coder windowCoder, + Coder keyCoder) throws IOException { + + int noOfKeys = reader.getInt(); + Map> activeTimers = new HashMap<>(noOfKeys); + activeTimers.clear(); + + CoderTypeSerializer keySerializer = new CoderTypeSerializer<>(keyCoder); + for (int i = 0; i < noOfKeys; i++) { + + // decode the key. + K key = reader.deserializeKey(keySerializer); + + // decode the associated timers. + Set timers = decodeTimerDataForKey(reader, windowCoder); + activeTimers.put(key, timers); + } + return activeTimers; + } + + private static void encodeTimerDataForKey(StateCheckpointWriter writer, Set timers) throws IOException { + // encode timers + writer.writeInt(timers.size()); + for (TimerInternals.TimerData timer : timers) { + String stringKey = timer.getNamespace().stringKey(); + + writer.setTag(stringKey); + writer.setTimestamp(timer.getTimestamp()); + writer.writeInt(timer.getDomain().ordinal()); + } + } + + private static Set decodeTimerDataForKey( + StateCheckpointReader reader, Coder windowCoder) throws IOException { + + // decode the timers: first their number and then the content itself. + int noOfTimers = reader.getInt(); + Set timers = new HashSet<>(noOfTimers); + for (int i = 0; i < noOfTimers; i++) { + String stringKey = reader.getTagToString(); + Instant instant = reader.getTimestamp(); + TimeDomain domain = TimeDomain.values()[reader.getInt()]; + + StateNamespace namespace = StateNamespaces.fromString(stringKey, windowCoder); + timers.add(TimerInternals.TimerData.of(namespace, instant, domain)); + } + return timers; + } } diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java index 1525c8012513..9f602fd71e50 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java @@ -25,103 +25,103 @@ public class StateCheckpointWriter { - private final AbstractStateBackend.CheckpointStateOutputView output; - - public static StateCheckpointWriter create(AbstractStateBackend.CheckpointStateOutputView output) { - return new StateCheckpointWriter(output); - } - - private StateCheckpointWriter(AbstractStateBackend.CheckpointStateOutputView output) { - this.output = output; - } - - ///////// Creating the serialized versions of the different types of state held by dataflow /////// - - public StateCheckpointWriter addValueBuilder() throws IOException { - validate(); - StateType.serialize(StateType.VALUE, this); - return this; - } - - public StateCheckpointWriter addWatermarkHoldsBuilder() throws IOException { - validate(); - StateType.serialize(StateType.WATERMARK, this); - return this; - } - - public StateCheckpointWriter addListUpdatesBuilder() throws IOException { - validate(); - StateType.serialize(StateType.LIST, this); - return this; - } - - public StateCheckpointWriter addAccumulatorBuilder() throws IOException { - validate(); - StateType.serialize(StateType.ACCUMULATOR, this); - return this; - } - - ///////// Setting the tag for a given state element /////// - - public StateCheckpointWriter setTag(ByteString stateKey) throws IOException { - return writeData(stateKey.toByteArray()); - } - - public StateCheckpointWriter setTag(String stateKey) throws IOException { - output.writeUTF(stateKey); - return this; - } - - - public StateCheckpointWriter serializeKey(K key, CoderTypeSerializer keySerializer) throws IOException { - return serializeObject(key, keySerializer); - } - - public StateCheckpointWriter serializeObject(T object, CoderTypeSerializer objectSerializer) throws IOException { - objectSerializer.serialize(object, output); - return this; - } - - ///////// Write the actual serialized data ////////// - - public StateCheckpointWriter setData(ByteString data) throws IOException { - return writeData(data.toByteArray()); - } - - public StateCheckpointWriter setData(byte[] data) throws IOException { - return writeData(data); - } - - public StateCheckpointWriter setTimestamp(Instant timestamp) throws IOException { - validate(); - output.writeLong(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis())); - return this; - } - - public StateCheckpointWriter writeInt(int number) throws IOException { - validate(); - output.writeInt(number); - return this; - } - - public StateCheckpointWriter writeByte(byte b) throws IOException { - validate(); - output.writeByte(b); - return this; - } - - ///////// Helper Methods /////// - - private StateCheckpointWriter writeData(byte[] data) throws IOException { - validate(); - output.writeInt(data.length); - output.write(data); - return this; - } - - private void validate() { - if (this.output == null) { - throw new RuntimeException("StateBackend not initialized yet."); - } - } + private final AbstractStateBackend.CheckpointStateOutputView output; + + public static StateCheckpointWriter create(AbstractStateBackend.CheckpointStateOutputView output) { + return new StateCheckpointWriter(output); + } + + private StateCheckpointWriter(AbstractStateBackend.CheckpointStateOutputView output) { + this.output = output; + } + + ///////// Creating the serialized versions of the different types of state held by dataflow /////// + + public StateCheckpointWriter addValueBuilder() throws IOException { + validate(); + StateType.serialize(StateType.VALUE, this); + return this; + } + + public StateCheckpointWriter addWatermarkHoldsBuilder() throws IOException { + validate(); + StateType.serialize(StateType.WATERMARK, this); + return this; + } + + public StateCheckpointWriter addListUpdatesBuilder() throws IOException { + validate(); + StateType.serialize(StateType.LIST, this); + return this; + } + + public StateCheckpointWriter addAccumulatorBuilder() throws IOException { + validate(); + StateType.serialize(StateType.ACCUMULATOR, this); + return this; + } + + ///////// Setting the tag for a given state element /////// + + public StateCheckpointWriter setTag(ByteString stateKey) throws IOException { + return writeData(stateKey.toByteArray()); + } + + public StateCheckpointWriter setTag(String stateKey) throws IOException { + output.writeUTF(stateKey); + return this; + } + + + public StateCheckpointWriter serializeKey(K key, CoderTypeSerializer keySerializer) throws IOException { + return serializeObject(key, keySerializer); + } + + public StateCheckpointWriter serializeObject(T object, CoderTypeSerializer objectSerializer) throws IOException { + objectSerializer.serialize(object, output); + return this; + } + + ///////// Write the actual serialized data ////////// + + public StateCheckpointWriter setData(ByteString data) throws IOException { + return writeData(data.toByteArray()); + } + + public StateCheckpointWriter setData(byte[] data) throws IOException { + return writeData(data); + } + + public StateCheckpointWriter setTimestamp(Instant timestamp) throws IOException { + validate(); + output.writeLong(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis())); + return this; + } + + public StateCheckpointWriter writeInt(int number) throws IOException { + validate(); + output.writeInt(number); + return this; + } + + public StateCheckpointWriter writeByte(byte b) throws IOException { + validate(); + output.writeByte(b); + return this; + } + + ///////// Helper Methods /////// + + private StateCheckpointWriter writeData(byte[] data) throws IOException { + validate(); + output.writeInt(data.length); + output.write(data); + return this; + } + + private void validate() { + if (this.output == null) { + throw new RuntimeException("StateBackend not initialized yet."); + } + } } \ No newline at end of file diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java index aa049efbacc3..9e2c9f8158a6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java @@ -23,49 +23,49 @@ * */ public enum StateType { - VALUE(0), + VALUE(0), - WATERMARK(1), + WATERMARK(1), - LIST(2), + LIST(2), - ACCUMULATOR(3); + ACCUMULATOR(3); - private final int numVal; + private final int numVal; - StateType(int value) { - this.numVal = value; - } + StateType(int value) { + this.numVal = value; + } - public static void serialize(StateType type, StateCheckpointWriter output) throws IOException { - if (output == null) { - throw new IllegalArgumentException("Cannot write to a null output."); - } + public static void serialize(StateType type, StateCheckpointWriter output) throws IOException { + if (output == null) { + throw new IllegalArgumentException("Cannot write to a null output."); + } - if(type.numVal < 0 || type.numVal > 3) { - throw new RuntimeException("Unknown State Type " + type + "."); - } + if(type.numVal < 0 || type.numVal > 3) { + throw new RuntimeException("Unknown State Type " + type + "."); + } - output.writeByte((byte) type.numVal); - } + output.writeByte((byte) type.numVal); + } - public static StateType deserialize(StateCheckpointReader input) throws IOException { - if (input == null) { - throw new IllegalArgumentException("Cannot read from a null input."); - } + public static StateType deserialize(StateCheckpointReader input) throws IOException { + if (input == null) { + throw new IllegalArgumentException("Cannot read from a null input."); + } - int typeInt = (int) input.getByte(); - if(typeInt < 0 || typeInt > 3) { - throw new RuntimeException("Unknown State Type " + typeInt + "."); - } + int typeInt = (int) input.getByte(); + if(typeInt < 0 || typeInt > 3) { + throw new RuntimeException("Unknown State Type " + typeInt + "."); + } - StateType resultType = null; - for(StateType st: values()) { - if(st.numVal == typeInt) { - resultType = st; - break; - } - } - return resultType; - } + StateType resultType = null; + for(StateType st: values()) { + if(st.numVal == typeInt) { + resultType = st; + break; + } + } + return resultType; + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java index ce53d44dc773..327297592679 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java @@ -30,70 +30,70 @@ public class AvroITCase extends JavaProgramTestBase { - protected String resultPath; - protected String tmpPath; - - public AvroITCase(){ - } - - static final String[] EXPECTED_RESULT = new String[] { - "Joe red 3", - "Mary blue 4", - "Mark green 1", - "Julia purple 5" - }; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - tmpPath = getTempDirPath("tmp"); - - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); - } - - @Override - protected void testProgram() throws Exception { - runProgram(tmpPath, resultPath); - } - - private static void runProgram(String tmpPath, String resultPath) { - Pipeline p = FlinkTestPipeline.createForBatch(); - - p - .apply(Create.of( - new User("Joe", 3, "red"), - new User("Mary", 4, "blue"), - new User("Mark", 1, "green"), - new User("Julia", 5, "purple")) - .withCoder(AvroCoder.of(User.class))) - - .apply(AvroIO.Write.to(tmpPath) - .withSchema(User.class)); - - p.run(); - - p = FlinkTestPipeline.createForBatch(); - - p - .apply(AvroIO.Read.from(tmpPath).withSchema(User.class).withoutValidation()) - - .apply(ParDo.of(new DoFn() { - @Override - public void processElement(ProcessContext c) throws Exception { - User u = c.element(); - String result = u.getName() + " " + u.getFavoriteColor() + " " + u.getFavoriteNumber(); - c.output(result); - } - })) - - .apply(TextIO.Write.to(resultPath)); - - p.run(); - } + protected String resultPath; + protected String tmpPath; + + public AvroITCase(){ + } + + static final String[] EXPECTED_RESULT = new String[] { + "Joe red 3", + "Mary blue 4", + "Mark green 1", + "Julia purple 5" + }; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + tmpPath = getTempDirPath("tmp"); + + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } + + @Override + protected void testProgram() throws Exception { + runProgram(tmpPath, resultPath); + } + + private static void runProgram(String tmpPath, String resultPath) { + Pipeline p = FlinkTestPipeline.createForBatch(); + + p + .apply(Create.of( + new User("Joe", 3, "red"), + new User("Mary", 4, "blue"), + new User("Mark", 1, "green"), + new User("Julia", 5, "purple")) + .withCoder(AvroCoder.of(User.class))) + + .apply(AvroIO.Write.to(tmpPath) + .withSchema(User.class)); + + p.run(); + + p = FlinkTestPipeline.createForBatch(); + + p + .apply(AvroIO.Read.from(tmpPath).withSchema(User.class).withoutValidation()) + + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + User u = c.element(); + String result = u.getName() + " " + u.getFavoriteColor() + " " + u.getFavoriteNumber(); + c.output(result); + } + })) + + .apply(TextIO.Write.to(resultPath)); + + p.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java index 928388cea011..e65e4977473e 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java @@ -26,47 +26,47 @@ public class FlattenizeITCase extends JavaProgramTestBase { - private String resultPath; - private String resultPath2; + private String resultPath; + private String resultPath2; - private static final String[] words = {"hello", "this", "is", "a", "DataSet!"}; - private static final String[] words2 = {"hello", "this", "is", "another", "DataSet!"}; - private static final String[] words3 = {"hello", "this", "is", "yet", "another", "DataSet!"}; + private static final String[] words = {"hello", "this", "is", "a", "DataSet!"}; + private static final String[] words2 = {"hello", "this", "is", "another", "DataSet!"}; + private static final String[] words3 = {"hello", "this", "is", "yet", "another", "DataSet!"}; - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - resultPath2 = getTempDirPath("result2"); - } + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + resultPath2 = getTempDirPath("result2"); + } - @Override - protected void postSubmit() throws Exception { - String join = Joiner.on('\n').join(words); - String join2 = Joiner.on('\n').join(words2); - String join3 = Joiner.on('\n').join(words3); - compareResultsByLinesInMemory(join + "\n" + join2, resultPath); - compareResultsByLinesInMemory(join + "\n" + join2 + "\n" + join3, resultPath2); - } + @Override + protected void postSubmit() throws Exception { + String join = Joiner.on('\n').join(words); + String join2 = Joiner.on('\n').join(words2); + String join3 = Joiner.on('\n').join(words3); + compareResultsByLinesInMemory(join + "\n" + join2, resultPath); + compareResultsByLinesInMemory(join + "\n" + join2 + "\n" + join3, resultPath2); + } - @Override - protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.createForBatch(); + @Override + protected void testProgram() throws Exception { + Pipeline p = FlinkTestPipeline.createForBatch(); - PCollection p1 = p.apply(Create.of(words)); - PCollection p2 = p.apply(Create.of(words2)); + PCollection p1 = p.apply(Create.of(words)); + PCollection p2 = p.apply(Create.of(words2)); - PCollectionList list = PCollectionList.of(p1).and(p2); + PCollectionList list = PCollectionList.of(p1).and(p2); - list.apply(Flatten.pCollections()).apply(TextIO.Write.to(resultPath)); + list.apply(Flatten.pCollections()).apply(TextIO.Write.to(resultPath)); - PCollection p3 = p.apply(Create.of(words3)); + PCollection p3 = p.apply(Create.of(words3)); - PCollectionList list2 = list.and(p3); + PCollectionList list2 = list.and(p3); - list2.apply(Flatten.pCollections()).apply(TextIO.Write.to(resultPath2)); + list2.apply(Flatten.pCollections()).apply(TextIO.Write.to(resultPath2)); - p.run(); - } + p.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java index 59c3b69998ce..578e0e142baa 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java @@ -26,45 +26,45 @@ */ public class FlinkTestPipeline extends Pipeline { - /** - * Creates and returns a new test pipeline for batch execution. - * - *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call - * {@link Pipeline#run} to execute the pipeline and check the tests. - */ - public static FlinkTestPipeline createForBatch() { - return create(false); - } + /** + * Creates and returns a new test pipeline for batch execution. + * + *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call + * {@link Pipeline#run} to execute the pipeline and check the tests. + */ + public static FlinkTestPipeline createForBatch() { + return create(false); + } - /** - * Creates and returns a new test pipeline for streaming execution. - * - *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call - * {@link Pipeline#run} to execute the pipeline and check the tests. - * - * @return The Test Pipeline - */ - public static FlinkTestPipeline createForStreaming() { - return create(true); - } + /** + * Creates and returns a new test pipeline for streaming execution. + * + *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call + * {@link Pipeline#run} to execute the pipeline and check the tests. + * + * @return The Test Pipeline + */ + public static FlinkTestPipeline createForStreaming() { + return create(true); + } - /** - * Creates and returns a new test pipeline for streaming or batch execution. - * - *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call - * {@link Pipeline#run} to execute the pipeline and check the tests. - * - * @param streaming True for streaming mode, False for batch. - * @return The Test Pipeline. - */ - private static FlinkTestPipeline create(boolean streaming) { - FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming); - return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions()); - } + /** + * Creates and returns a new test pipeline for streaming or batch execution. + * + *

    Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call + * {@link Pipeline#run} to execute the pipeline and check the tests. + * + * @param streaming True for streaming mode, False for batch. + * @return The Test Pipeline. + */ + private static FlinkTestPipeline create(boolean streaming) { + FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming); + return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions()); + } - private FlinkTestPipeline(PipelineRunner runner, - PipelineOptions options) { - super(runner, options); - } + private FlinkTestPipeline(PipelineRunner runner, + PipelineOptions options) { + super(runner, options); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java index af0f21779157..28861eaa27e4 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java @@ -34,66 +34,66 @@ */ public class JoinExamplesITCase extends JavaProgramTestBase { - protected String resultPath; - - public JoinExamplesITCase(){ - } - - private static final TableRow row1 = new TableRow() - .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212") - .set("Actor1Name", "BANGKOK").set("SOURCEURL", "http://cnn.com"); - private static final TableRow row2 = new TableRow() - .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212") - .set("Actor1Name", "LAOS").set("SOURCEURL", "http://www.chicagotribune.com"); - private static final TableRow row3 = new TableRow() - .set("ActionGeo_CountryCode", "BE").set("SQLDATE", "20141213") - .set("Actor1Name", "AFGHANISTAN").set("SOURCEURL", "http://cnn.com"); - static final TableRow[] EVENTS = new TableRow[] { - row1, row2, row3 - }; - static final List EVENT_ARRAY = Arrays.asList(EVENTS); - - private static final TableRow cc1 = new TableRow() - .set("FIPSCC", "VM").set("HumanName", "Vietnam"); - private static final TableRow cc2 = new TableRow() - .set("FIPSCC", "BE").set("HumanName", "Belgium"); - static final TableRow[] CCS = new TableRow[] { - cc1, cc2 - }; - static final List CC_ARRAY = Arrays.asList(CCS); - - static final String[] JOINED_EVENTS = new String[] { - "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: LAOS, " - + "url: http://www.chicagotribune.com", - "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: BANGKOK, " - + "url: http://cnn.com", - "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, " - + "url: http://cnn.com" - }; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(JOINED_EVENTS), resultPath); - } - - @Override - protected void testProgram() throws Exception { - - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection input1 = p.apply(Create.of(EVENT_ARRAY)); - PCollection input2 = p.apply(Create.of(CC_ARRAY)); - - PCollection output = JoinExamples.joinEvents(input1, input2); - - output.apply(TextIO.Write.to(resultPath)); - - p.run(); - } + protected String resultPath; + + public JoinExamplesITCase(){ + } + + private static final TableRow row1 = new TableRow() + .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212") + .set("Actor1Name", "BANGKOK").set("SOURCEURL", "http://cnn.com"); + private static final TableRow row2 = new TableRow() + .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212") + .set("Actor1Name", "LAOS").set("SOURCEURL", "http://www.chicagotribune.com"); + private static final TableRow row3 = new TableRow() + .set("ActionGeo_CountryCode", "BE").set("SQLDATE", "20141213") + .set("Actor1Name", "AFGHANISTAN").set("SOURCEURL", "http://cnn.com"); + static final TableRow[] EVENTS = new TableRow[] { + row1, row2, row3 + }; + static final List EVENT_ARRAY = Arrays.asList(EVENTS); + + private static final TableRow cc1 = new TableRow() + .set("FIPSCC", "VM").set("HumanName", "Vietnam"); + private static final TableRow cc2 = new TableRow() + .set("FIPSCC", "BE").set("HumanName", "Belgium"); + static final TableRow[] CCS = new TableRow[] { + cc1, cc2 + }; + static final List CC_ARRAY = Arrays.asList(CCS); + + static final String[] JOINED_EVENTS = new String[] { + "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: LAOS, " + + "url: http://www.chicagotribune.com", + "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: BANGKOK, " + + "url: http://cnn.com", + "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, " + + "url: http://cnn.com" + }; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(JOINED_EVENTS), resultPath); + } + + @Override + protected void testProgram() throws Exception { + + Pipeline p = FlinkTestPipeline.createForBatch(); + + PCollection input1 = p.apply(Create.of(EVENT_ARRAY)); + PCollection input2 = p.apply(Create.of(CC_ARRAY)); + + PCollection output = JoinExamples.joinEvents(input1, input2); + + output.apply(TextIO.Write.to(resultPath)); + + p.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java index 35f2eaff463d..d1652e7b4ffe 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java @@ -27,37 +27,37 @@ public class MaybeEmptyTestITCase extends JavaProgramTestBase implements Serializable { - protected String resultPath; - - protected final String expected = "test"; - - public MaybeEmptyTestITCase() { - } - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(expected, resultPath); - } - - @Override - protected void testProgram() throws Exception { - - Pipeline p = FlinkTestPipeline.createForBatch(); - - p.apply(Create.of((Void) null)).setCoder(VoidCoder.of()) - .apply(ParDo.of( - new DoFn() { - @Override - public void processElement(DoFn.ProcessContext c) { - c.output(expected); - } - })).apply(TextIO.Write.to(resultPath)); - p.run(); - } + protected String resultPath; + + protected final String expected = "test"; + + public MaybeEmptyTestITCase() { + } + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expected, resultPath); + } + + @Override + protected void testProgram() throws Exception { + + Pipeline p = FlinkTestPipeline.createForBatch(); + + p.apply(Create.of((Void) null)).setCoder(VoidCoder.of()) + .apply(ParDo.of( + new DoFn() { + @Override + public void processElement(DoFn.ProcessContext c) { + c.output(expected); + } + })).apply(TextIO.Write.to(resultPath)); + p.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java index ccdbbf90b134..d8087d6563ab 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java @@ -31,68 +31,68 @@ public class ParDoMultiOutputITCase extends JavaProgramTestBase implements Serializable { - private String resultPath; - - private static String[] expectedWords = {"MAAA", "MAAFOOO"}; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on("\n").join(expectedWords), resultPath); - } - - @Override - protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO")); - - // Select words whose length is below a cut off, - // plus the lengths of words that are above the cut off. - // Also select words starting with "MARKER". - final int wordLengthCutOff = 3; - // Create tags to use for the main and side outputs. - final TupleTag wordsBelowCutOffTag = new TupleTag(){}; - final TupleTag wordLengthsAboveCutOffTag = new TupleTag(){}; - final TupleTag markedWordsTag = new TupleTag(){}; - - PCollectionTuple results = - words.apply(ParDo - .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag) - .and(markedWordsTag)) - .of(new DoFn() { - final TupleTag specialWordsTag = new TupleTag() { - }; - - public void processElement(ProcessContext c) { - String word = c.element(); - if (word.length() <= wordLengthCutOff) { - c.output(word); - } else { - c.sideOutput(wordLengthsAboveCutOffTag, word.length()); - } - if (word.startsWith("MAA")) { - c.sideOutput(markedWordsTag, word); - } - - if (word.startsWith("SPECIAL")) { - c.sideOutput(specialWordsTag, word); - } - } - })); - - // Extract the PCollection results, by tag. - PCollection wordsBelowCutOff = results.get(wordsBelowCutOffTag); - PCollection wordLengthsAboveCutOff = results.get - (wordLengthsAboveCutOffTag); - PCollection markedWords = results.get(markedWordsTag); - - markedWords.apply(TextIO.Write.to(resultPath)); - - p.run(); - } + private String resultPath; + + private static String[] expectedWords = {"MAAA", "MAAFOOO"}; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on("\n").join(expectedWords), resultPath); + } + + @Override + protected void testProgram() throws Exception { + Pipeline p = FlinkTestPipeline.createForBatch(); + + PCollection words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO")); + + // Select words whose length is below a cut off, + // plus the lengths of words that are above the cut off. + // Also select words starting with "MARKER". + final int wordLengthCutOff = 3; + // Create tags to use for the main and side outputs. + final TupleTag wordsBelowCutOffTag = new TupleTag(){}; + final TupleTag wordLengthsAboveCutOffTag = new TupleTag(){}; + final TupleTag markedWordsTag = new TupleTag(){}; + + PCollectionTuple results = + words.apply(ParDo + .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag) + .and(markedWordsTag)) + .of(new DoFn() { + final TupleTag specialWordsTag = new TupleTag() { + }; + + public void processElement(ProcessContext c) { + String word = c.element(); + if (word.length() <= wordLengthCutOff) { + c.output(word); + } else { + c.sideOutput(wordLengthsAboveCutOffTag, word.length()); + } + if (word.startsWith("MAA")) { + c.sideOutput(markedWordsTag, word); + } + + if (word.startsWith("SPECIAL")) { + c.sideOutput(specialWordsTag, word); + } + } + })); + + // Extract the PCollection results, by tag. + PCollection wordsBelowCutOff = results.get(wordsBelowCutOffTag); + PCollection wordLengthsAboveCutOff = results.get + (wordLengthsAboveCutOffTag); + PCollection markedWords = results.get(markedWordsTag); + + markedWords.apply(TextIO.Write.to(resultPath)); + + p.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java index 3569a788ed90..5a463590f715 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java @@ -36,128 +36,128 @@ public class ReadSourceITCase extends JavaProgramTestBase { - protected String resultPath; - - public ReadSourceITCase(){ - } - - static final String[] EXPECTED_RESULT = new String[] { - "1", "2", "3", "4", "5", "6", "7", "8", "9"}; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); - } - - @Override - protected void testProgram() throws Exception { - runProgram(resultPath); - } - - private static void runProgram(String resultPath) { - - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection result = p - .apply(Read.from(new ReadSource(1, 10))) - .apply(ParDo.of(new DoFn() { - @Override - public void processElement(ProcessContext c) throws Exception { - c.output(c.element().toString()); - } - })); - - result.apply(TextIO.Write.to(resultPath)); - p.run(); - } - - - private static class ReadSource extends BoundedSource { - final int from; - final int to; - - ReadSource(int from, int to) { - this.from = from; - this.to = to; - } - - @Override - public List splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options) - throws Exception { - List res = new ArrayList<>(); - FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class); - int numWorkers = flinkOptions.getParallelism(); - Preconditions.checkArgument(numWorkers > 0, "Number of workers should be larger than 0."); - - float step = 1.0f * (to - from) / numWorkers; - for (int i = 0; i < numWorkers; ++i) { - res.add(new ReadSource(Math.round(from + i * step), Math.round(from + (i + 1) * step))); - } - return res; - } - - @Override - public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { - return 8 * (to - from); - } - - @Override - public boolean producesSortedKeys(PipelineOptions options) throws Exception { - return true; - } - - @Override - public BoundedReader createReader(PipelineOptions options) throws IOException { - return new RangeReader(this); - } - - @Override - public void validate() {} - - @Override - public Coder getDefaultOutputCoder() { - return BigEndianIntegerCoder.of(); - } - - private class RangeReader extends BoundedReader { - private int current; - - public RangeReader(ReadSource source) { - this.current = source.from - 1; - } - - @Override - public boolean start() throws IOException { - return true; - } - - @Override - public boolean advance() throws IOException { - current++; - return (current < to); - } - - @Override - public Integer getCurrent() { - return current; - } - - @Override - public void close() throws IOException { - // Nothing - } - - @Override - public BoundedSource getCurrentSource() { - return ReadSource.this; - } - } - } + protected String resultPath; + + public ReadSourceITCase(){ + } + + static final String[] EXPECTED_RESULT = new String[] { + "1", "2", "3", "4", "5", "6", "7", "8", "9"}; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } + + @Override + protected void testProgram() throws Exception { + runProgram(resultPath); + } + + private static void runProgram(String resultPath) { + + Pipeline p = FlinkTestPipeline.createForBatch(); + + PCollection result = p + .apply(Read.from(new ReadSource(1, 10))) + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + c.output(c.element().toString()); + } + })); + + result.apply(TextIO.Write.to(resultPath)); + p.run(); + } + + + private static class ReadSource extends BoundedSource { + final int from; + final int to; + + ReadSource(int from, int to) { + this.from = from; + this.to = to; + } + + @Override + public List splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options) + throws Exception { + List res = new ArrayList<>(); + FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class); + int numWorkers = flinkOptions.getParallelism(); + Preconditions.checkArgument(numWorkers > 0, "Number of workers should be larger than 0."); + + float step = 1.0f * (to - from) / numWorkers; + for (int i = 0; i < numWorkers; ++i) { + res.add(new ReadSource(Math.round(from + i * step), Math.round(from + (i + 1) * step))); + } + return res; + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return 8 * (to - from); + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) throws Exception { + return true; + } + + @Override + public BoundedReader createReader(PipelineOptions options) throws IOException { + return new RangeReader(this); + } + + @Override + public void validate() {} + + @Override + public Coder getDefaultOutputCoder() { + return BigEndianIntegerCoder.of(); + } + + private class RangeReader extends BoundedReader { + private int current; + + public RangeReader(ReadSource source) { + this.current = source.from - 1; + } + + @Override + public boolean start() throws IOException { + return true; + } + + @Override + public boolean advance() throws IOException { + current++; + return (current < to); + } + + @Override + public Integer getCurrent() { + return current; + } + + @Override + public void close() throws IOException { + // Nothing + } + + @Override + public BoundedSource getCurrentSource() { + return ReadSource.this; + } + } + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java index db794f73da75..615f19420a84 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java @@ -30,39 +30,39 @@ public class RemoveDuplicatesEmptyITCase extends JavaProgramTestBase { - protected String resultPath; + protected String resultPath; - public RemoveDuplicatesEmptyITCase(){ - } + public RemoveDuplicatesEmptyITCase(){ + } - static final String[] EXPECTED_RESULT = new String[] {}; + static final String[] EXPECTED_RESULT = new String[] {}; - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); - } + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } - @Override - protected void testProgram() throws Exception { + @Override + protected void testProgram() throws Exception { - List strings = Collections.emptyList(); + List strings = Collections.emptyList(); - Pipeline p = FlinkTestPipeline.createForBatch(); + Pipeline p = FlinkTestPipeline.createForBatch(); - PCollection input = - p.apply(Create.of(strings)) - .setCoder(StringUtf8Coder.of()); + PCollection input = + p.apply(Create.of(strings)) + .setCoder(StringUtf8Coder.of()); - PCollection output = - input.apply(RemoveDuplicates.create()); + PCollection output = + input.apply(RemoveDuplicates.create()); - output.apply(TextIO.Write.to(resultPath)); - p.run(); - } + output.apply(TextIO.Write.to(resultPath)); + p.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java index 04e06b88118d..8c19f2cb6c50 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java @@ -30,40 +30,40 @@ public class RemoveDuplicatesITCase extends JavaProgramTestBase { - protected String resultPath; + protected String resultPath; - public RemoveDuplicatesITCase(){ - } + public RemoveDuplicatesITCase(){ + } - static final String[] EXPECTED_RESULT = new String[] { - "k1", "k5", "k2", "k3"}; + static final String[] EXPECTED_RESULT = new String[] { + "k1", "k5", "k2", "k3"}; - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); - } + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } - @Override - protected void testProgram() throws Exception { + @Override + protected void testProgram() throws Exception { - List strings = Arrays.asList("k1", "k5", "k5", "k2", "k1", "k2", "k3"); + List strings = Arrays.asList("k1", "k5", "k5", "k2", "k1", "k2", "k3"); - Pipeline p = FlinkTestPipeline.createForBatch(); + Pipeline p = FlinkTestPipeline.createForBatch(); - PCollection input = - p.apply(Create.of(strings)) - .setCoder(StringUtf8Coder.of()); + PCollection input = + p.apply(Create.of(strings)) + .setCoder(StringUtf8Coder.of()); - PCollection output = - input.apply(RemoveDuplicates.create()); + PCollection output = + input.apply(RemoveDuplicates.create()); - output.apply(TextIO.Write.to(resultPath)); - p.run(); - } + output.apply(TextIO.Write.to(resultPath)); + p.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java index ee8843c5e8da..7c3d6f936859 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java @@ -28,40 +28,40 @@ public class SideInputITCase extends JavaProgramTestBase implements Serializable { - private static final String expected = "Hello!"; + private static final String expected = "Hello!"; - protected String resultPath; + protected String resultPath; - @Override - protected void testProgram() throws Exception { + @Override + protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.createForBatch(); + Pipeline p = FlinkTestPipeline.createForBatch(); - final PCollectionView sidesInput = p - .apply(Create.of(expected)) - .apply(View.asSingleton()); + final PCollectionView sidesInput = p + .apply(Create.of(expected)) + .apply(View.asSingleton()); - p.apply(Create.of("bli")) - .apply(ParDo.of(new DoFn() { - @Override - public void processElement(ProcessContext c) throws Exception { - String s = c.sideInput(sidesInput); - c.output(s); - } - }).withSideInputs(sidesInput)).apply(TextIO.Write.to(resultPath)); + p.apply(Create.of("bli")) + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + String s = c.sideInput(sidesInput); + c.output(s); + } + }).withSideInputs(sidesInput)).apply(TextIO.Write.to(resultPath)); - p.run(); - } + p.run(); + } - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(expected, resultPath); - } + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(expected, resultPath); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java index 07c1294f52e6..715d0be7704a 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java @@ -32,45 +32,45 @@ public class TfIdfITCase extends JavaProgramTestBase { - protected String resultPath; + protected String resultPath; - public TfIdfITCase(){ - } + public TfIdfITCase(){ + } - static final String[] EXPECTED_RESULT = new String[] { - "a", "m", "n", "b", "c", "d"}; + static final String[] EXPECTED_RESULT = new String[] { + "a", "m", "n", "b", "c", "d"}; - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); - } + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } - @Override - protected void testProgram() throws Exception { + @Override + protected void testProgram() throws Exception { - Pipeline pipeline = FlinkTestPipeline.createForBatch(); + Pipeline pipeline = FlinkTestPipeline.createForBatch(); - pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); + pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class)); - PCollection>> wordToUriAndTfIdf = pipeline - .apply(Create.of( - KV.of(new URI("x"), "a b c d"), - KV.of(new URI("y"), "a b c"), - KV.of(new URI("z"), "a m n"))) - .apply(new TFIDF.ComputeTfIdf()); + PCollection>> wordToUriAndTfIdf = pipeline + .apply(Create.of( + KV.of(new URI("x"), "a b c d"), + KV.of(new URI("y"), "a b c"), + KV.of(new URI("z"), "a m n"))) + .apply(new TFIDF.ComputeTfIdf()); - PCollection words = wordToUriAndTfIdf - .apply(Keys.create()) - .apply(RemoveDuplicates.create()); + PCollection words = wordToUriAndTfIdf + .apply(Keys.create()) + .apply(RemoveDuplicates.create()); - words.apply(TextIO.Write.to(resultPath)); + words.apply(TextIO.Write.to(resultPath)); - pipeline.run(); - } + pipeline.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java index 91880979b430..f1a2454c16a2 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java @@ -32,43 +32,43 @@ public class WordCountITCase extends JavaProgramTestBase { - protected String resultPath; + protected String resultPath; - public WordCountITCase(){ - } + public WordCountITCase(){ + } - static final String[] WORDS_ARRAY = new String[] { - "hi there", "hi", "hi sue bob", - "hi sue", "", "bob hi"}; + static final String[] WORDS_ARRAY = new String[] { + "hi there", "hi", "hi sue bob", + "hi sue", "", "bob hi"}; - static final List WORDS = Arrays.asList(WORDS_ARRAY); + static final List WORDS = Arrays.asList(WORDS_ARRAY); - static final String[] COUNTS_ARRAY = new String[] { - "hi: 5", "there: 1", "sue: 2", "bob: 2"}; + static final String[] COUNTS_ARRAY = new String[] { + "hi: 5", "there: 1", "sue: 2", "bob: 2"}; - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(COUNTS_ARRAY), resultPath); - } + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(COUNTS_ARRAY), resultPath); + } - @Override - protected void testProgram() throws Exception { + @Override + protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.createForBatch(); + Pipeline p = FlinkTestPipeline.createForBatch(); - PCollection input = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); + PCollection input = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of()); - input - .apply(new WordCount.CountWords()) - .apply(MapElements.via(new WordCount.FormatAsTextFn())) - .apply(TextIO.Write.to(resultPath)); + input + .apply(new WordCount.CountWords()) + .apply(MapElements.via(new WordCount.FormatAsTextFn())) + .apply(TextIO.Write.to(resultPath)); - p.run(); - } + p.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java index ccc52c43b3ea..1cac036e1026 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java @@ -33,104 +33,104 @@ public class WordCountJoin2ITCase extends JavaProgramTestBase { - static final String[] WORDS_1 = new String[] { - "hi there", "hi", "hi sue bob", - "hi sue", "", "bob hi"}; - - static final String[] WORDS_2 = new String[] { - "hi tim", "beauty", "hooray sue bob", - "hi there", "", "please say hi"}; - - static final String[] RESULTS = new String[] { - "beauty -> Tag1: Tag2: 1", - "bob -> Tag1: 2 Tag2: 1", - "hi -> Tag1: 5 Tag2: 3", - "hooray -> Tag1: Tag2: 1", - "please -> Tag1: Tag2: 1", - "say -> Tag1: Tag2: 1", - "sue -> Tag1: 2 Tag2: 1", - "there -> Tag1: 1 Tag2: 1", - "tim -> Tag1: Tag2: 1" - }; - - static final TupleTag tag1 = new TupleTag<>("Tag1"); - static final TupleTag tag2 = new TupleTag<>("Tag2"); - - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath); - } - - @Override - protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.createForBatch(); - - /* Create two PCollections and join them */ - PCollection> occurences1 = p.apply(Create.of(WORDS_1)) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); - - PCollection> occurences2 = p.apply(Create.of(WORDS_2)) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); - - /* CoGroup the two collections */ - PCollection> mergedOccurences = KeyedPCollectionTuple - .of(tag1, occurences1) - .and(tag2, occurences2) - .apply(CoGroupByKey.create()); - - /* Format output */ - mergedOccurences.apply(ParDo.of(new FormatCountsFn())) - .apply(TextIO.Write.named("test").to(resultPath)); - - p.run(); - } - - - static class ExtractWordsFn extends DoFn { - - @Override - public void startBundle(Context c) { - } - - @Override - public void processElement(ProcessContext c) { - // Split the line into words. - String[] words = c.element().split("[^a-zA-Z']+"); - - // Output each word encountered into the output PCollection. - for (String word : words) { - if (!word.isEmpty()) { - c.output(word); - } - } - } - } - - static class FormatCountsFn extends DoFn, String> { - @Override - public void processElement(ProcessContext c) { - CoGbkResult value = c.element().getValue(); - String key = c.element().getKey(); - String countTag1 = tag1.getId() + ": "; - String countTag2 = tag2.getId() + ": "; - for (Long count : value.getAll(tag1)) { - countTag1 += count + " "; - } - for (Long count : value.getAll(tag2)) { - countTag2 += count; - } - c.output(key + " -> " + countTag1 + countTag2); - } - } + static final String[] WORDS_1 = new String[] { + "hi there", "hi", "hi sue bob", + "hi sue", "", "bob hi"}; + + static final String[] WORDS_2 = new String[] { + "hi tim", "beauty", "hooray sue bob", + "hi there", "", "please say hi"}; + + static final String[] RESULTS = new String[] { + "beauty -> Tag1: Tag2: 1", + "bob -> Tag1: 2 Tag2: 1", + "hi -> Tag1: 5 Tag2: 3", + "hooray -> Tag1: Tag2: 1", + "please -> Tag1: Tag2: 1", + "say -> Tag1: Tag2: 1", + "sue -> Tag1: 2 Tag2: 1", + "there -> Tag1: 1 Tag2: 1", + "tim -> Tag1: Tag2: 1" + }; + + static final TupleTag tag1 = new TupleTag<>("Tag1"); + static final TupleTag tag2 = new TupleTag<>("Tag2"); + + protected String resultPath; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath); + } + + @Override + protected void testProgram() throws Exception { + Pipeline p = FlinkTestPipeline.createForBatch(); + + /* Create two PCollections and join them */ + PCollection> occurences1 = p.apply(Create.of(WORDS_1)) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Count.perElement()); + + PCollection> occurences2 = p.apply(Create.of(WORDS_2)) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Count.perElement()); + + /* CoGroup the two collections */ + PCollection> mergedOccurences = KeyedPCollectionTuple + .of(tag1, occurences1) + .and(tag2, occurences2) + .apply(CoGroupByKey.create()); + + /* Format output */ + mergedOccurences.apply(ParDo.of(new FormatCountsFn())) + .apply(TextIO.Write.named("test").to(resultPath)); + + p.run(); + } + + + static class ExtractWordsFn extends DoFn { + + @Override + public void startBundle(Context c) { + } + + @Override + public void processElement(ProcessContext c) { + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + static class FormatCountsFn extends DoFn, String> { + @Override + public void processElement(ProcessContext c) { + CoGbkResult value = c.element().getValue(); + String key = c.element().getKey(); + String countTag1 = tag1.getId() + ": "; + String countTag2 = tag2.getId() + ": "; + for (Long count : value.getAll(tag1)) { + countTag1 += count + " "; + } + for (Long count : value.getAll(tag2)) { + countTag2 += count; + } + c.output(key + " -> " + countTag1 + countTag2); + } + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java index e6eddc0c6a55..4c8b99b3c484 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java @@ -33,122 +33,122 @@ public class WordCountJoin3ITCase extends JavaProgramTestBase { - static final String[] WORDS_1 = new String[] { - "hi there", "hi", "hi sue bob", - "hi sue", "", "bob hi"}; - - static final String[] WORDS_2 = new String[] { - "hi tim", "beauty", "hooray sue bob", - "hi there", "", "please say hi"}; - - static final String[] WORDS_3 = new String[] { - "hi stephan", "beauty", "hooray big fabian", - "hi yo", "", "please say hi"}; - - static final String[] RESULTS = new String[] { - "beauty -> Tag1: Tag2: 1 Tag3: 1", - "bob -> Tag1: 2 Tag2: 1 Tag3: ", - "hi -> Tag1: 5 Tag2: 3 Tag3: 3", - "hooray -> Tag1: Tag2: 1 Tag3: 1", - "please -> Tag1: Tag2: 1 Tag3: 1", - "say -> Tag1: Tag2: 1 Tag3: 1", - "sue -> Tag1: 2 Tag2: 1 Tag3: ", - "there -> Tag1: 1 Tag2: 1 Tag3: ", - "tim -> Tag1: Tag2: 1 Tag3: ", - "stephan -> Tag1: Tag2: Tag3: 1", - "yo -> Tag1: Tag2: Tag3: 1", - "fabian -> Tag1: Tag2: Tag3: 1", - "big -> Tag1: Tag2: Tag3: 1" - }; - - static final TupleTag tag1 = new TupleTag<>("Tag1"); - static final TupleTag tag2 = new TupleTag<>("Tag2"); - static final TupleTag tag3 = new TupleTag<>("Tag3"); - - protected String resultPath; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath); - } - - @Override - protected void testProgram() throws Exception { - - Pipeline p = FlinkTestPipeline.createForBatch(); - - /* Create two PCollections and join them */ - PCollection> occurences1 = p.apply(Create.of(WORDS_1)) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); - - PCollection> occurences2 = p.apply(Create.of(WORDS_2)) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); - - PCollection> occurences3 = p.apply(Create.of(WORDS_3)) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Count.perElement()); - - /* CoGroup the two collections */ - PCollection> mergedOccurences = KeyedPCollectionTuple - .of(tag1, occurences1) - .and(tag2, occurences2) - .and(tag3, occurences3) - .apply(CoGroupByKey.create()); - - /* Format output */ - mergedOccurences.apply(ParDo.of(new FormatCountsFn())) - .apply(TextIO.Write.named("test").to(resultPath)); - - p.run(); - } - - - static class ExtractWordsFn extends DoFn { - - @Override - public void startBundle(Context c) { - } - - @Override - public void processElement(ProcessContext c) { - // Split the line into words. - String[] words = c.element().split("[^a-zA-Z']+"); - - // Output each word encountered into the output PCollection. - for (String word : words) { - if (!word.isEmpty()) { - c.output(word); - } - } - } - } - - static class FormatCountsFn extends DoFn, String> { - @Override - public void processElement(ProcessContext c) { - CoGbkResult value = c.element().getValue(); - String key = c.element().getKey(); - String countTag1 = tag1.getId() + ": "; - String countTag2 = tag2.getId() + ": "; - String countTag3 = tag3.getId() + ": "; - for (Long count : value.getAll(tag1)) { - countTag1 += count + " "; - } - for (Long count : value.getAll(tag2)) { - countTag2 += count + " "; - } - for (Long count : value.getAll(tag3)) { - countTag3 += count; - } - c.output(key + " -> " + countTag1 + countTag2 + countTag3); - } - } + static final String[] WORDS_1 = new String[] { + "hi there", "hi", "hi sue bob", + "hi sue", "", "bob hi"}; + + static final String[] WORDS_2 = new String[] { + "hi tim", "beauty", "hooray sue bob", + "hi there", "", "please say hi"}; + + static final String[] WORDS_3 = new String[] { + "hi stephan", "beauty", "hooray big fabian", + "hi yo", "", "please say hi"}; + + static final String[] RESULTS = new String[] { + "beauty -> Tag1: Tag2: 1 Tag3: 1", + "bob -> Tag1: 2 Tag2: 1 Tag3: ", + "hi -> Tag1: 5 Tag2: 3 Tag3: 3", + "hooray -> Tag1: Tag2: 1 Tag3: 1", + "please -> Tag1: Tag2: 1 Tag3: 1", + "say -> Tag1: Tag2: 1 Tag3: 1", + "sue -> Tag1: 2 Tag2: 1 Tag3: ", + "there -> Tag1: 1 Tag2: 1 Tag3: ", + "tim -> Tag1: Tag2: 1 Tag3: ", + "stephan -> Tag1: Tag2: Tag3: 1", + "yo -> Tag1: Tag2: Tag3: 1", + "fabian -> Tag1: Tag2: Tag3: 1", + "big -> Tag1: Tag2: Tag3: 1" + }; + + static final TupleTag tag1 = new TupleTag<>("Tag1"); + static final TupleTag tag2 = new TupleTag<>("Tag2"); + static final TupleTag tag3 = new TupleTag<>("Tag3"); + + protected String resultPath; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath); + } + + @Override + protected void testProgram() throws Exception { + + Pipeline p = FlinkTestPipeline.createForBatch(); + + /* Create two PCollections and join them */ + PCollection> occurences1 = p.apply(Create.of(WORDS_1)) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Count.perElement()); + + PCollection> occurences2 = p.apply(Create.of(WORDS_2)) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Count.perElement()); + + PCollection> occurences3 = p.apply(Create.of(WORDS_3)) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Count.perElement()); + + /* CoGroup the two collections */ + PCollection> mergedOccurences = KeyedPCollectionTuple + .of(tag1, occurences1) + .and(tag2, occurences2) + .and(tag3, occurences3) + .apply(CoGroupByKey.create()); + + /* Format output */ + mergedOccurences.apply(ParDo.of(new FormatCountsFn())) + .apply(TextIO.Write.named("test").to(resultPath)); + + p.run(); + } + + + static class ExtractWordsFn extends DoFn { + + @Override + public void startBundle(Context c) { + } + + @Override + public void processElement(ProcessContext c) { + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + static class FormatCountsFn extends DoFn, String> { + @Override + public void processElement(ProcessContext c) { + CoGbkResult value = c.element().getValue(); + String key = c.element().getKey(); + String countTag1 = tag1.getId() + ": "; + String countTag2 = tag2.getId() + ": "; + String countTag3 = tag3.getId() + ": "; + for (Long count : value.getAll(tag1)) { + countTag1 += count + " "; + } + for (Long count : value.getAll(tag2)) { + countTag2 += count + " "; + } + for (Long count : value.getAll(tag3)) { + countTag3 += count; + } + c.output(key + " -> " + countTag1 + countTag2 + countTag3); + } + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java index 865fc5f93894..a61bf52b2f99 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java @@ -39,118 +39,118 @@ */ public class WriteSinkITCase extends JavaProgramTestBase { - protected String resultPath; + protected String resultPath; - public WriteSinkITCase(){ - } + public WriteSinkITCase(){ + } - static final String[] EXPECTED_RESULT = new String[] { - "Joe red 3", "Mary blue 4", "Max yellow 23"}; + static final String[] EXPECTED_RESULT = new String[] { + "Joe red 3", "Mary blue 4", "Max yellow 23"}; - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); - } - - @Override - protected void testProgram() throws Exception { - runProgram(resultPath); - } - - private static void runProgram(String resultPath) { - Pipeline p = FlinkTestPipeline.createForBatch(); - - p.apply(Create.of(EXPECTED_RESULT)).setCoder(StringUtf8Coder.of()) - .apply("CustomSink", Write.to(new MyCustomSink(resultPath))); - - p.run(); - } - - /** - * Simple custom sink which writes to a file. - */ - private static class MyCustomSink extends Sink { - - private final String resultPath; - - public MyCustomSink(String resultPath) { - this.resultPath = resultPath; - } - - @Override - public void validate(PipelineOptions options) { - assertNotNull(options); - } - - @Override - public WriteOperation createWriteOperation(PipelineOptions options) { - return new MyWriteOperation(); - } - - private class MyWriteOperation extends WriteOperation { - - @Override - public Coder getWriterResultCoder() { - return StringUtf8Coder.of(); - } - - @Override - public void initialize(PipelineOptions options) throws Exception { - - } - - @Override - public void finalize(Iterable writerResults, PipelineOptions options) throws Exception { - - } - - @Override - public Writer createWriter(PipelineOptions options) throws Exception { - return new MyWriter(); - } - - @Override - public Sink getSink() { - return MyCustomSink.this; - } - - /** - * Simple Writer which writes to a file. - */ - private class MyWriter extends Writer { - - private PrintWriter internalWriter; - - @Override - public void open(String uId) throws Exception { - Path path = new Path(resultPath + "/" + uId); - FileSystem.get(new URI("file:///")).create(path, false); - internalWriter = new PrintWriter(new File(path.toUri())); - } - - @Override - public void write(String value) throws Exception { - internalWriter.println(value); - } - - @Override - public String close() throws Exception { - internalWriter.close(); - return resultPath; - } - - @Override - public WriteOperation getWriteOperation() { - return MyWriteOperation.this; - } - } - } - } + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } + + @Override + protected void testProgram() throws Exception { + runProgram(resultPath); + } + + private static void runProgram(String resultPath) { + Pipeline p = FlinkTestPipeline.createForBatch(); + + p.apply(Create.of(EXPECTED_RESULT)).setCoder(StringUtf8Coder.of()) + .apply("CustomSink", Write.to(new MyCustomSink(resultPath))); + + p.run(); + } + + /** + * Simple custom sink which writes to a file. + */ + private static class MyCustomSink extends Sink { + + private final String resultPath; + + public MyCustomSink(String resultPath) { + this.resultPath = resultPath; + } + + @Override + public void validate(PipelineOptions options) { + assertNotNull(options); + } + + @Override + public WriteOperation createWriteOperation(PipelineOptions options) { + return new MyWriteOperation(); + } + + private class MyWriteOperation extends WriteOperation { + + @Override + public Coder getWriterResultCoder() { + return StringUtf8Coder.of(); + } + + @Override + public void initialize(PipelineOptions options) throws Exception { + + } + + @Override + public void finalize(Iterable writerResults, PipelineOptions options) throws Exception { + + } + + @Override + public Writer createWriter(PipelineOptions options) throws Exception { + return new MyWriter(); + } + + @Override + public Sink getSink() { + return MyCustomSink.this; + } + + /** + * Simple Writer which writes to a file. + */ + private class MyWriter extends Writer { + + private PrintWriter internalWriter; + + @Override + public void open(String uId) throws Exception { + Path path = new Path(resultPath + "/" + uId); + FileSystem.get(new URI("file:///")).create(path, false); + internalWriter = new PrintWriter(new File(path.toUri())); + } + + @Override + public void write(String value) throws Exception { + internalWriter.println(value); + } + + @Override + public String close() throws Exception { + internalWriter.close(); + return resultPath; + } + + @Override + public WriteOperation getWriteOperation() { + return MyWriteOperation.this; + } + } + } + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java index 01f9c323d52e..fb240f4412fb 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java @@ -43,464 +43,464 @@ public class GroupAlsoByWindowTest { - private final Combine.CombineFn combiner = new Sum.SumIntegerFn(); - - private final WindowingStrategy slidingWindowWithAfterWatermarkTriggerStrategy = - WindowingStrategy.of(SlidingWindows.of(Duration.standardSeconds(10)).every(Duration.standardSeconds(5))) - .withTrigger(AfterWatermark.pastEndOfWindow()).withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES); - - private final WindowingStrategy sessionWindowingStrategy = - WindowingStrategy.of(Sessions.withGapDuration(Duration.standardSeconds(2))) - .withTrigger(Repeatedly.forever(AfterWatermark.pastEndOfWindow())) - .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES) - .withAllowedLateness(Duration.standardSeconds(100)); - - private final WindowingStrategy fixedWindowingStrategy = - WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(10))); - - private final WindowingStrategy fixedWindowWithCountTriggerStrategy = - fixedWindowingStrategy.withTrigger(AfterPane.elementCountAtLeast(5)); - - private final WindowingStrategy fixedWindowWithAfterWatermarkTriggerStrategy = - fixedWindowingStrategy.withTrigger(AfterWatermark.pastEndOfWindow()); - - private final WindowingStrategy fixedWindowWithCompoundTriggerStrategy = - fixedWindowingStrategy.withTrigger( - AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(5)) - .withLateFirings(AfterPane.elementCountAtLeast(5)).buildTrigger()); - - /** - * The default accumulation mode is - * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#DISCARDING_FIRED_PANES}. - * This strategy changes it to - * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#ACCUMULATING_FIRED_PANES} - */ - private final WindowingStrategy fixedWindowWithCompoundTriggerStrategyAcc = - fixedWindowWithCompoundTriggerStrategy - .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES); - - @Test - public void testWithLateness() throws Exception { - WindowingStrategy strategy = WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(2))) - .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES) - .withAllowedLateness(Duration.millis(1000)); - long initialTime = 0L; - Pipeline pipeline = FlinkTestPipeline.createForStreaming(); - - KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); - - FlinkGroupAlsoByWindowWrapper gbwOperaror = - FlinkGroupAlsoByWindowWrapper.createForTesting( - pipeline.getOptions(), - pipeline.getCoderRegistry(), - strategy, - inputCoder, - combiner.asKeyedFn()); - - OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = - new OneInputStreamOperatorTestHarness<>(gbwOperaror); - testHarness.open(); - - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processWatermark(new Watermark(initialTime + 2000)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processWatermark(new Watermark(initialTime + 4000)); - - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 4), - new Instant(initialTime + 1), - new IntervalWindow(new Instant(0), new Instant(2000)), - PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) - , initialTime + 1)); - expectedOutput.add(new Watermark(initialTime + 2000)); - - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 5), - new Instant(initialTime + 1999), - new IntervalWindow(new Instant(0), new Instant(2000)), - PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)) - , initialTime + 1999)); - - - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 6), - new Instant(initialTime + 1999), - new IntervalWindow(new Instant(0), new Instant(2000)), - PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 2, 2)) - , initialTime + 1999)); - expectedOutput.add(new Watermark(initialTime + 4000)); - - TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); - testHarness.close(); - } - - @Test - public void testSessionWindows() throws Exception { - WindowingStrategy strategy = sessionWindowingStrategy; - - long initialTime = 0L; - Pipeline pipeline = FlinkTestPipeline.createForStreaming(); - - KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); - - FlinkGroupAlsoByWindowWrapper gbwOperaror = - FlinkGroupAlsoByWindowWrapper.createForTesting( - pipeline.getOptions(), - pipeline.getCoderRegistry(), - strategy, - inputCoder, - combiner.asKeyedFn()); - - OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = - new OneInputStreamOperatorTestHarness<>(gbwOperaror); - testHarness.open(); - - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3500), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3700), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 2700), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processWatermark(new Watermark(initialTime + 6000)); - - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6700), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6800), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 8900), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 7600), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 5600), null, PaneInfo.NO_FIRING), initialTime + 20)); - - testHarness.processWatermark(new Watermark(initialTime + 12000)); - - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 6), - new Instant(initialTime + 1), - new IntervalWindow(new Instant(1), new Instant(5700)), - PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) - , initialTime + 1)); - expectedOutput.add(new Watermark(initialTime + 6000)); - - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 11), - new Instant(initialTime + 6700), - new IntervalWindow(new Instant(1), new Instant(10900)), - PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) - , initialTime + 6700)); - expectedOutput.add(new Watermark(initialTime + 12000)); - - TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); - testHarness.close(); - } - - @Test - public void testSlidingWindows() throws Exception { - WindowingStrategy strategy = slidingWindowWithAfterWatermarkTriggerStrategy; - long initialTime = 0L; - OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = - createTestingOperatorAndState(strategy, initialTime); - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); - testHarness.processWatermark(new Watermark(initialTime + 25000)); - - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 6), - new Instant(initialTime + 5000), - new IntervalWindow(new Instant(0), new Instant(10000)), - PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime + 5000)); - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 6), - new Instant(initialTime + 1), - new IntervalWindow(new Instant(-5000), new Instant(5000)), - PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime + 1)); - expectedOutput.add(new Watermark(initialTime + 10000)); - - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 11), - new Instant(initialTime + 15000), - new IntervalWindow(new Instant(10000), new Instant(20000)), - PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime + 15000)); - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 3), - new Instant(initialTime + 10000), - new IntervalWindow(new Instant(5000), new Instant(15000)), - PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime + 10000)); - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key2", 1), - new Instant(initialTime + 19500), - new IntervalWindow(new Instant(10000), new Instant(20000)), - PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime + 19500)); - expectedOutput.add(new Watermark(initialTime + 20000)); - - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key2", 1), - new Instant(initialTime + 20000), - /** - * this is 20000 and not 19500 because of a convention in dataflow where - * timestamps of windowed values in a window cannot be smaller than the - * end of a previous window. Checkout the documentation of the - * {@link WindowFn#getOutputTime(Instant, BoundedWindow)} - */ - new IntervalWindow(new Instant(15000), new Instant(25000)), - PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime + 20000)); - expectedOutput.add(new StreamRecord<>( - WindowedValue.of(KV.of("key1", 8), - new Instant(initialTime + 20000), - new IntervalWindow(new Instant(15000), new Instant(25000)), - PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) - , initialTime + 20000)); - expectedOutput.add(new Watermark(initialTime + 25000)); - - TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); - testHarness.close(); - } - - @Test - public void testAfterWatermarkProgram() throws Exception { - WindowingStrategy strategy = fixedWindowWithAfterWatermarkTriggerStrategy; - long initialTime = 0L; - OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = - createTestingOperatorAndState(strategy, initialTime); - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); - - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6), - new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 1)); - expectedOutput.add(new Watermark(initialTime + 10000)); - - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11), - new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 10000)); - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500)); - expectedOutput.add(new Watermark(initialTime + 20000)); - - TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); - testHarness.close(); - } - - @Test - public void testAfterCountProgram() throws Exception { - WindowingStrategy strategy = fixedWindowWithCountTriggerStrategy; - - long initialTime = 0L; - OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = - createTestingOperatorAndState(strategy, initialTime); - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); - - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 1)); - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 10000)); - expectedOutput.add(new Watermark(initialTime + 10000)); - - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime + 19500)); - expectedOutput.add(new Watermark(initialTime + 20000)); - TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); - - testHarness.close(); - } - - @Test - public void testCompoundProgram() throws Exception { - WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategy; - - long initialTime = 0L; - OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = - createTestingOperatorAndState(strategy, initialTime); - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); - - /** - * PaneInfo are: - * isFirst (pane in window), - * isLast, Timing (of triggering), - * index (of pane in the window), - * onTimeIndex (if it the 1st,2nd, ... pane that was fired on time) - * */ - - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1)); - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000)); - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500)); - - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), - new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200)); - - expectedOutput.add(new Watermark(initialTime + 10000)); - - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500)); - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500)); - - expectedOutput.add(new Watermark(initialTime + 20000)); - TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); - - testHarness.close(); - } - - @Test - public void testCompoundAccumulatingPanesProgram() throws Exception { - WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc; - long initialTime = 0L; - OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = - createTestingOperatorAndState(strategy, initialTime); - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); - - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1)); - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), - new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000)); - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 10), - new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500)); - - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6), - new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200)); - - expectedOutput.add(new Watermark(initialTime + 10000)); - - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11), - new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500)); - expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), - new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500)); - - expectedOutput.add(new Watermark(initialTime + 20000)); - TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); - - testHarness.close(); - } - - private OneInputStreamOperatorTestHarness createTestingOperatorAndState(WindowingStrategy strategy, long initialTime) throws Exception { - Pipeline pipeline = FlinkTestPipeline.createForStreaming(); - - KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); - - FlinkGroupAlsoByWindowWrapper gbwOperaror = - FlinkGroupAlsoByWindowWrapper.createForTesting( - pipeline.getOptions(), - pipeline.getCoderRegistry(), - strategy, - inputCoder, - combiner.asKeyedFn()); - - OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = - new OneInputStreamOperatorTestHarness<>(gbwOperaror); - testHarness.open(); - - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); - - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 10000), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 12100), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 14200), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 15300), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 16500), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); - - testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); - - testHarness.processWatermark(new Watermark(initialTime + 10000)); - testHarness.processWatermark(new Watermark(initialTime + 20000)); - - return testHarness; - } - - private static class ResultSortComparator implements Comparator { - @Override - public int compare(Object o1, Object o2) { - if (o1 instanceof Watermark && o2 instanceof Watermark) { - Watermark w1 = (Watermark) o1; - Watermark w2 = (Watermark) o2; - return (int) (w1.getTimestamp() - w2.getTimestamp()); - } else { - StreamRecord>> sr0 = (StreamRecord>>) o1; - StreamRecord>> sr1 = (StreamRecord>>) o2; - - int comparison = (int) (sr0.getValue().getTimestamp().getMillis() - sr1.getValue().getTimestamp().getMillis()); - if (comparison != 0) { - return comparison; - } - - comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey()); - if(comparison == 0) { - comparison = Integer.compare( - sr0.getValue().getValue().getValue(), - sr1.getValue().getValue().getValue()); - } - if(comparison == 0) { - Collection windowsA = sr0.getValue().getWindows(); - Collection windowsB = sr1.getValue().getWindows(); - - if(windowsA.size() != 1 || windowsB.size() != 1) { - throw new IllegalStateException("A value cannot belong to more than one windows after grouping."); - } - - BoundedWindow windowA = (BoundedWindow) windowsA.iterator().next(); - BoundedWindow windowB = (BoundedWindow) windowsB.iterator().next(); - comparison = Long.compare(windowA.maxTimestamp().getMillis(), windowB.maxTimestamp().getMillis()); - } - return comparison; - } - } - } - - private WindowedValue makeWindowedValue(WindowingStrategy strategy, - T output, Instant timestamp, Collection windows, PaneInfo pane) { - final Instant inputTimestamp = timestamp; - final WindowFn windowFn = strategy.getWindowFn(); - - if (timestamp == null) { - timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; - } - - if (windows == null) { - try { - windows = windowFn.assignWindows(windowFn.new AssignContext() { - @Override - public Object element() { - throw new UnsupportedOperationException( - "WindowFn attempted to access input element when none was available"); - } - - @Override - public Instant timestamp() { - if (inputTimestamp == null) { - throw new UnsupportedOperationException( - "WindowFn attempted to access input timestamp when none was available"); - } - return inputTimestamp; - } - - @Override - public Collection windows() { - throw new UnsupportedOperationException( - "WindowFn attempted to access input windows when none were available"); - } - }); - } catch (Exception e) { - throw UserCodeException.wrap(e); - } - } - - return WindowedValue.of(output, timestamp, windows, pane); - } + private final Combine.CombineFn combiner = new Sum.SumIntegerFn(); + + private final WindowingStrategy slidingWindowWithAfterWatermarkTriggerStrategy = + WindowingStrategy.of(SlidingWindows.of(Duration.standardSeconds(10)).every(Duration.standardSeconds(5))) + .withTrigger(AfterWatermark.pastEndOfWindow()).withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES); + + private final WindowingStrategy sessionWindowingStrategy = + WindowingStrategy.of(Sessions.withGapDuration(Duration.standardSeconds(2))) + .withTrigger(Repeatedly.forever(AfterWatermark.pastEndOfWindow())) + .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES) + .withAllowedLateness(Duration.standardSeconds(100)); + + private final WindowingStrategy fixedWindowingStrategy = + WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(10))); + + private final WindowingStrategy fixedWindowWithCountTriggerStrategy = + fixedWindowingStrategy.withTrigger(AfterPane.elementCountAtLeast(5)); + + private final WindowingStrategy fixedWindowWithAfterWatermarkTriggerStrategy = + fixedWindowingStrategy.withTrigger(AfterWatermark.pastEndOfWindow()); + + private final WindowingStrategy fixedWindowWithCompoundTriggerStrategy = + fixedWindowingStrategy.withTrigger( + AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(5)) + .withLateFirings(AfterPane.elementCountAtLeast(5)).buildTrigger()); + + /** + * The default accumulation mode is + * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#DISCARDING_FIRED_PANES}. + * This strategy changes it to + * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#ACCUMULATING_FIRED_PANES} + */ + private final WindowingStrategy fixedWindowWithCompoundTriggerStrategyAcc = + fixedWindowWithCompoundTriggerStrategy + .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES); + + @Test + public void testWithLateness() throws Exception { + WindowingStrategy strategy = WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(2))) + .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES) + .withAllowedLateness(Duration.millis(1000)); + long initialTime = 0L; + Pipeline pipeline = FlinkTestPipeline.createForStreaming(); + + KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); + + FlinkGroupAlsoByWindowWrapper gbwOperaror = + FlinkGroupAlsoByWindowWrapper.createForTesting( + pipeline.getOptions(), + pipeline.getCoderRegistry(), + strategy, + inputCoder, + combiner.asKeyedFn()); + + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + new OneInputStreamOperatorTestHarness<>(gbwOperaror); + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processWatermark(new Watermark(initialTime + 2000)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processWatermark(new Watermark(initialTime + 4000)); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 4), + new Instant(initialTime + 1), + new IntervalWindow(new Instant(0), new Instant(2000)), + PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) + , initialTime + 1)); + expectedOutput.add(new Watermark(initialTime + 2000)); + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 5), + new Instant(initialTime + 1999), + new IntervalWindow(new Instant(0), new Instant(2000)), + PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)) + , initialTime + 1999)); + + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 6), + new Instant(initialTime + 1999), + new IntervalWindow(new Instant(0), new Instant(2000)), + PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 2, 2)) + , initialTime + 1999)); + expectedOutput.add(new Watermark(initialTime + 4000)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + testHarness.close(); + } + + @Test + public void testSessionWindows() throws Exception { + WindowingStrategy strategy = sessionWindowingStrategy; + + long initialTime = 0L; + Pipeline pipeline = FlinkTestPipeline.createForStreaming(); + + KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); + + FlinkGroupAlsoByWindowWrapper gbwOperaror = + FlinkGroupAlsoByWindowWrapper.createForTesting( + pipeline.getOptions(), + pipeline.getCoderRegistry(), + strategy, + inputCoder, + combiner.asKeyedFn()); + + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + new OneInputStreamOperatorTestHarness<>(gbwOperaror); + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3700), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 2700), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processWatermark(new Watermark(initialTime + 6000)); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6700), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6800), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 8900), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 7600), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 5600), null, PaneInfo.NO_FIRING), initialTime + 20)); + + testHarness.processWatermark(new Watermark(initialTime + 12000)); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 6), + new Instant(initialTime + 1), + new IntervalWindow(new Instant(1), new Instant(5700)), + PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) + , initialTime + 1)); + expectedOutput.add(new Watermark(initialTime + 6000)); + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 11), + new Instant(initialTime + 6700), + new IntervalWindow(new Instant(1), new Instant(10900)), + PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0)) + , initialTime + 6700)); + expectedOutput.add(new Watermark(initialTime + 12000)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + testHarness.close(); + } + + @Test + public void testSlidingWindows() throws Exception { + WindowingStrategy strategy = slidingWindowWithAfterWatermarkTriggerStrategy; + long initialTime = 0L; + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + createTestingOperatorAndState(strategy, initialTime); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + testHarness.processWatermark(new Watermark(initialTime + 25000)); + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 6), + new Instant(initialTime + 5000), + new IntervalWindow(new Instant(0), new Instant(10000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime + 5000)); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 6), + new Instant(initialTime + 1), + new IntervalWindow(new Instant(-5000), new Instant(5000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime + 1)); + expectedOutput.add(new Watermark(initialTime + 10000)); + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 11), + new Instant(initialTime + 15000), + new IntervalWindow(new Instant(10000), new Instant(20000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime + 15000)); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 3), + new Instant(initialTime + 10000), + new IntervalWindow(new Instant(5000), new Instant(15000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime + 10000)); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key2", 1), + new Instant(initialTime + 19500), + new IntervalWindow(new Instant(10000), new Instant(20000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime + 19500)); + expectedOutput.add(new Watermark(initialTime + 20000)); + + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key2", 1), + new Instant(initialTime + 20000), + /** + * this is 20000 and not 19500 because of a convention in dataflow where + * timestamps of windowed values in a window cannot be smaller than the + * end of a previous window. Checkout the documentation of the + * {@link WindowFn#getOutputTime(Instant, BoundedWindow)} + */ + new IntervalWindow(new Instant(15000), new Instant(25000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime + 20000)); + expectedOutput.add(new StreamRecord<>( + WindowedValue.of(KV.of("key1", 8), + new Instant(initialTime + 20000), + new IntervalWindow(new Instant(15000), new Instant(25000)), + PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)) + , initialTime + 20000)); + expectedOutput.add(new Watermark(initialTime + 25000)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + testHarness.close(); + } + + @Test + public void testAfterWatermarkProgram() throws Exception { + WindowingStrategy strategy = fixedWindowWithAfterWatermarkTriggerStrategy; + long initialTime = 0L; + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + createTestingOperatorAndState(strategy, initialTime); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6), + new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 1)); + expectedOutput.add(new Watermark(initialTime + 10000)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11), + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 10000)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500)); + expectedOutput.add(new Watermark(initialTime + 20000)); + + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + testHarness.close(); + } + + @Test + public void testAfterCountProgram() throws Exception { + WindowingStrategy strategy = fixedWindowWithCountTriggerStrategy; + + long initialTime = 0L; + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + createTestingOperatorAndState(strategy, initialTime); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 1)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 10000)); + expectedOutput.add(new Watermark(initialTime + 10000)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime + 19500)); + expectedOutput.add(new Watermark(initialTime + 20000)); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + + testHarness.close(); + } + + @Test + public void testCompoundProgram() throws Exception { + WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategy; + + long initialTime = 0L; + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + createTestingOperatorAndState(strategy, initialTime); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + /** + * PaneInfo are: + * isFirst (pane in window), + * isLast, Timing (of triggering), + * index (of pane in the window), + * onTimeIndex (if it the 1st,2nd, ... pane that was fired on time) + * */ + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), + new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200)); + + expectedOutput.add(new Watermark(initialTime + 10000)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500)); + + expectedOutput.add(new Watermark(initialTime + 20000)); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + + testHarness.close(); + } + + @Test + public void testCompoundAccumulatingPanesProgram() throws Exception { + WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc; + long initialTime = 0L; + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + createTestingOperatorAndState(strategy, initialTime); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5), + new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 10), + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6), + new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200)); + + expectedOutput.add(new Watermark(initialTime + 10000)); + + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11), + new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500)); + expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), + new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500)); + + expectedOutput.add(new Watermark(initialTime + 20000)); + TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator()); + + testHarness.close(); + } + + private OneInputStreamOperatorTestHarness createTestingOperatorAndState(WindowingStrategy strategy, long initialTime) throws Exception { + Pipeline pipeline = FlinkTestPipeline.createForStreaming(); + + KvCoder inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()); + + FlinkGroupAlsoByWindowWrapper gbwOperaror = + FlinkGroupAlsoByWindowWrapper.createForTesting( + pipeline.getOptions(), + pipeline.getCoderRegistry(), + strategy, + inputCoder, + combiner.asKeyedFn()); + + OneInputStreamOperatorTestHarness>, WindowedValue>> testHarness = + new OneInputStreamOperatorTestHarness<>(gbwOperaror); + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20)); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 10000), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 12100), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 14200), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 15300), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 16500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + + testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20)); + + testHarness.processWatermark(new Watermark(initialTime + 10000)); + testHarness.processWatermark(new Watermark(initialTime + 20000)); + + return testHarness; + } + + private static class ResultSortComparator implements Comparator { + @Override + public int compare(Object o1, Object o2) { + if (o1 instanceof Watermark && o2 instanceof Watermark) { + Watermark w1 = (Watermark) o1; + Watermark w2 = (Watermark) o2; + return (int) (w1.getTimestamp() - w2.getTimestamp()); + } else { + StreamRecord>> sr0 = (StreamRecord>>) o1; + StreamRecord>> sr1 = (StreamRecord>>) o2; + + int comparison = (int) (sr0.getValue().getTimestamp().getMillis() - sr1.getValue().getTimestamp().getMillis()); + if (comparison != 0) { + return comparison; + } + + comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey()); + if(comparison == 0) { + comparison = Integer.compare( + sr0.getValue().getValue().getValue(), + sr1.getValue().getValue().getValue()); + } + if(comparison == 0) { + Collection windowsA = sr0.getValue().getWindows(); + Collection windowsB = sr1.getValue().getWindows(); + + if(windowsA.size() != 1 || windowsB.size() != 1) { + throw new IllegalStateException("A value cannot belong to more than one windows after grouping."); + } + + BoundedWindow windowA = (BoundedWindow) windowsA.iterator().next(); + BoundedWindow windowB = (BoundedWindow) windowsB.iterator().next(); + comparison = Long.compare(windowA.maxTimestamp().getMillis(), windowB.maxTimestamp().getMillis()); + } + return comparison; + } + } + } + + private WindowedValue makeWindowedValue(WindowingStrategy strategy, + T output, Instant timestamp, Collection windows, PaneInfo pane) { + final Instant inputTimestamp = timestamp; + final WindowFn windowFn = strategy.getWindowFn(); + + if (timestamp == null) { + timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + if (windows == null) { + try { + windows = windowFn.assignWindows(windowFn.new AssignContext() { + @Override + public Object element() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input element when none was available"); + } + + @Override + public Instant timestamp() { + if (inputTimestamp == null) { + throw new UnsupportedOperationException( + "WindowFn attempted to access input timestamp when none was available"); + } + return inputTimestamp; + } + + @Override + public Collection windows() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input windows when none were available"); + } + }); + } catch (Exception e) { + throw UserCodeException.wrap(e); + } + } + + return WindowedValue.of(output, timestamp, windows, pane); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java index 5a412aa54098..52e9e25b442e 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java @@ -39,83 +39,83 @@ public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable { - protected String resultPath; + protected String resultPath; - static final String[] EXPECTED_RESULT = new String[] { - "k: null v: user1 user1 user1 user2 user2 user2 user2 user3" - }; + static final String[] EXPECTED_RESULT = new String[] { + "k: null v: user1 user1 user1 user2 user2 user2 user2 user3" + }; - public GroupByNullKeyTest(){ - } + public GroupByNullKeyTest(){ + } - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); - } + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } - public static class ExtractUserAndTimestamp extends DoFn, String> { - private static final long serialVersionUID = 0; + public static class ExtractUserAndTimestamp extends DoFn, String> { + private static final long serialVersionUID = 0; - @Override - public void processElement(ProcessContext c) { - KV record = c.element(); - long now = System.currentTimeMillis(); - int timestamp = record.getKey(); - String userName = record.getValue(); - if (userName != null) { - // Sets the implicit timestamp field to be used in windowing. - c.outputWithTimestamp(userName, new Instant(timestamp + now)); - } - } - } + @Override + public void processElement(ProcessContext c) { + KV record = c.element(); + long now = System.currentTimeMillis(); + int timestamp = record.getKey(); + String userName = record.getValue(); + if (userName != null) { + // Sets the implicit timestamp field to be used in windowing. + c.outputWithTimestamp(userName, new Instant(timestamp + now)); + } + } + } - @Override - protected void testProgram() throws Exception { + @Override + protected void testProgram() throws Exception { - Pipeline p = FlinkTestPipeline.createForStreaming(); + Pipeline p = FlinkTestPipeline.createForStreaming(); - PCollection output = - p.apply(Create.of(Arrays.asList( - KV.of(0, "user1"), - KV.of(1, "user1"), - KV.of(2, "user1"), - KV.of(10, "user2"), - KV.of(1, "user2"), - KV.of(15000, "user2"), - KV.of(12000, "user2"), - KV.of(25000, "user3")))) - .apply(ParDo.of(new ExtractUserAndTimestamp())) - .apply(Window.into(FixedWindows.of(Duration.standardHours(1))) - .triggering(AfterWatermark.pastEndOfWindow()) - .withAllowedLateness(Duration.ZERO) - .discardingFiredPanes()) + PCollection output = + p.apply(Create.of(Arrays.asList( + KV.of(0, "user1"), + KV.of(1, "user1"), + KV.of(2, "user1"), + KV.of(10, "user2"), + KV.of(1, "user2"), + KV.of(15000, "user2"), + KV.of(12000, "user2"), + KV.of(25000, "user3")))) + .apply(ParDo.of(new ExtractUserAndTimestamp())) + .apply(Window.into(FixedWindows.of(Duration.standardHours(1))) + .triggering(AfterWatermark.pastEndOfWindow()) + .withAllowedLateness(Duration.ZERO) + .discardingFiredPanes()) - .apply(ParDo.of(new DoFn>() { - @Override - public void processElement(ProcessContext c) throws Exception { - String elem = c.element(); - c.output(KV.of((Void) null, elem)); - } - })) - .apply(GroupByKey.create()) - .apply(ParDo.of(new DoFn>, String>() { - @Override - public void processElement(ProcessContext c) throws Exception { - KV> elem = c.element(); - StringBuilder str = new StringBuilder(); - str.append("k: " + elem.getKey() + " v:"); - for (String v : elem.getValue()) { - str.append(" " + v); - } - c.output(str.toString()); - } - })); - output.apply(TextIO.Write.to(resultPath)); - p.run(); - } + .apply(ParDo.of(new DoFn>() { + @Override + public void processElement(ProcessContext c) throws Exception { + String elem = c.element(); + c.output(KV.of((Void) null, elem)); + } + })) + .apply(GroupByKey.create()) + .apply(ParDo.of(new DoFn>, String>() { + @Override + public void processElement(ProcessContext c) throws Exception { + KV> elem = c.element(); + StringBuilder str = new StringBuilder(); + str.append("k: " + elem.getKey() + " v:"); + for (String v : elem.getValue()) { + str.append(" " + v); + } + c.output(str.toString()); + } + })); + output.apply(TextIO.Write.to(resultPath)); + p.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java index 7489fcc46bf6..d5b10437d778 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java @@ -43,261 +43,261 @@ public class StateSerializationTest { - private static final StateNamespace NAMESPACE_1 = StateNamespaces.global(); - private static final String KEY_PREFIX = "TEST_"; - - // TODO: This can be replaced with the standard Sum.SumIntererFn once the state no longer needs - // to create a StateTag at the point of restoring state. Currently StateTags are compared strictly - // by type and combiners always use KeyedCombineFnWithContext rather than KeyedCombineFn or CombineFn. - private static CombineWithContext.KeyedCombineFnWithContext SUM_COMBINER = - new CombineWithContext.KeyedCombineFnWithContext() { - @Override - public int[] createAccumulator(Object key, CombineWithContext.Context c) { - return new int[1]; - } - - @Override - public int[] addInput(Object key, int[] accumulator, Integer value, CombineWithContext.Context c) { - accumulator[0] += value; - return accumulator; - } - - @Override - public int[] mergeAccumulators(Object key, Iterable accumulators, CombineWithContext.Context c) { - int[] r = new int[1]; - for (int[] a : accumulators) { - r[0] += a[0]; - } - return r; - } - - @Override - public Integer extractOutput(Object key, int[] accumulator, CombineWithContext.Context c) { - return accumulator[0]; - } - }; - - private static Coder INT_ACCUM_CODER = DelegateCoder.of( - VarIntCoder.of(), - new DelegateCoder.CodingFunction() { - @Override - public Integer apply(int[] accumulator) { - return accumulator[0]; - } - }, - new DelegateCoder.CodingFunction() { - @Override - public int[] apply(Integer value) { - int[] a = new int[1]; - a[0] = value; - return a; - } - }); - - private static final StateTag> STRING_VALUE_ADDR = - StateTags.value("stringValue", StringUtf8Coder.of()); - private static final StateTag> INT_VALUE_ADDR = - StateTags.value("stringValue", VarIntCoder.of()); - private static final StateTag> SUM_INTEGER_ADDR = - StateTags.keyedCombiningValueWithContext("sumInteger", INT_ACCUM_CODER, SUM_COMBINER); - private static final StateTag> STRING_BAG_ADDR = - StateTags.bag("stringBag", StringUtf8Coder.of()); - private static final StateTag> WATERMARK_BAG_ADDR = - StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); - - private Map> statePerKey = new HashMap<>(); - - private Map> activeTimers = new HashMap<>(); - - private void initializeStateAndTimers() throws CannotProvideCoderException { - for (int i = 0; i < 10; i++) { - String key = KEY_PREFIX + i; - - FlinkStateInternals state = initializeStateForKey(key); - Set timers = new HashSet<>(); - for (int j = 0; j < 5; j++) { - TimerInternals.TimerData timer = TimerInternals - .TimerData.of(NAMESPACE_1, - new Instant(1000 + i + j), TimeDomain.values()[j % 3]); - timers.add(timer); - } - - statePerKey.put(key, state); - activeTimers.put(key, timers); - } - } - - private FlinkStateInternals initializeStateForKey(String key) throws CannotProvideCoderException { - FlinkStateInternals state = createState(key); - - ValueState value = state.state(NAMESPACE_1, STRING_VALUE_ADDR); - value.write("test"); - - ValueState value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR); - value2.write(4); - value2.write(5); - - AccumulatorCombiningState combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR); - combiningValue.add(1); - combiningValue.add(2); - - WatermarkHoldState watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR); - watermark.add(new Instant(1000)); - - BagState bag = state.state(NAMESPACE_1, STRING_BAG_ADDR); - bag.add("v1"); - bag.add("v2"); - bag.add("v3"); - bag.add("v4"); - return state; - } - - private boolean restoreAndTestState(DataInputView in) throws Exception { - StateCheckpointReader reader = new StateCheckpointReader(in); - final ClassLoader userClassloader = this.getClass().getClassLoader(); - Coder windowCoder = IntervalWindow.getCoder(); - Coder keyCoder = StringUtf8Coder.of(); - - boolean comparisonRes = true; - - for (String key : statePerKey.keySet()) { - comparisonRes &= checkStateForKey(key); - } - - // restore the timers - Map> restoredTimersPerKey = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder); - if (activeTimers.size() != restoredTimersPerKey.size()) { - return false; - } - - for (String key : statePerKey.keySet()) { - Set originalTimers = activeTimers.get(key); - Set restoredTimers = restoredTimersPerKey.get(key); - comparisonRes &= checkTimersForKey(originalTimers, restoredTimers); - } - - // restore the state - Map> restoredPerKeyState = - StateCheckpointUtils.decodeState(reader, OutputTimeFns.outputAtEarliestInputTimestamp(), keyCoder, windowCoder, userClassloader); - if (restoredPerKeyState.size() != statePerKey.size()) { - return false; - } - - for (String key : statePerKey.keySet()) { - FlinkStateInternals originalState = statePerKey.get(key); - FlinkStateInternals restoredState = restoredPerKeyState.get(key); - comparisonRes &= checkStateForKey(originalState, restoredState); - } - return comparisonRes; - } - - private boolean checkStateForKey(String key) throws CannotProvideCoderException { - FlinkStateInternals state = statePerKey.get(key); - - ValueState value = state.state(NAMESPACE_1, STRING_VALUE_ADDR); - boolean comp = value.read().equals("test"); - - ValueState value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR); - comp &= value2.read().equals(5); - - AccumulatorCombiningState combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR); - comp &= combiningValue.read().equals(3); - - WatermarkHoldState watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR); - comp &= watermark.read().equals(new Instant(1000)); - - BagState bag = state.state(NAMESPACE_1, STRING_BAG_ADDR); - Iterator it = bag.read().iterator(); - int i = 0; - while (it.hasNext()) { - comp &= it.next().equals("v" + (++i)); - } - return comp; - } - - private void storeState(AbstractStateBackend.CheckpointStateOutputView out) throws Exception { - StateCheckpointWriter checkpointBuilder = StateCheckpointWriter.create(out); - Coder keyCoder = StringUtf8Coder.of(); - - // checkpoint the timers - StateCheckpointUtils.encodeTimers(activeTimers, checkpointBuilder, keyCoder); - - // checkpoint the state - StateCheckpointUtils.encodeState(statePerKey, checkpointBuilder, keyCoder); - } - - private boolean checkTimersForKey(Set originalTimers, Set restoredTimers) { - boolean comp = true; - if (restoredTimers == null) { - return false; - } - - if (originalTimers.size() != restoredTimers.size()) { - return false; - } - - for (TimerInternals.TimerData timer : originalTimers) { - comp &= restoredTimers.contains(timer); - } - return comp; - } - - private boolean checkStateForKey(FlinkStateInternals originalState, FlinkStateInternals restoredState) throws CannotProvideCoderException { - if (restoredState == null) { - return false; - } - - ValueState orValue = originalState.state(NAMESPACE_1, STRING_VALUE_ADDR); - ValueState resValue = restoredState.state(NAMESPACE_1, STRING_VALUE_ADDR); - boolean comp = orValue.read().equals(resValue.read()); - - ValueState orIntValue = originalState.state(NAMESPACE_1, INT_VALUE_ADDR); - ValueState resIntValue = restoredState.state(NAMESPACE_1, INT_VALUE_ADDR); - comp &= orIntValue.read().equals(resIntValue.read()); - - AccumulatorCombiningState combOrValue = originalState.state(NAMESPACE_1, SUM_INTEGER_ADDR); - AccumulatorCombiningState combResValue = restoredState.state(NAMESPACE_1, SUM_INTEGER_ADDR); - comp &= combOrValue.read().equals(combResValue.read()); - - WatermarkHoldState orWatermark = originalState.state(NAMESPACE_1, WATERMARK_BAG_ADDR); - WatermarkHoldState resWatermark = restoredState.state(NAMESPACE_1, WATERMARK_BAG_ADDR); - comp &= orWatermark.read().equals(resWatermark.read()); - - BagState orBag = originalState.state(NAMESPACE_1, STRING_BAG_ADDR); - BagState resBag = restoredState.state(NAMESPACE_1, STRING_BAG_ADDR); - - Iterator orIt = orBag.read().iterator(); - Iterator resIt = resBag.read().iterator(); - - while (orIt.hasNext() && resIt.hasNext()) { - comp &= orIt.next().equals(resIt.next()); - } - - return !((orIt.hasNext() && !resIt.hasNext()) || (!orIt.hasNext() && resIt.hasNext())) && comp; - } - - private FlinkStateInternals createState(String key) throws CannotProvideCoderException { - return new FlinkStateInternals<>( - key, - StringUtf8Coder.of(), - IntervalWindow.getCoder(), - OutputTimeFns.outputAtEarliestInputTimestamp()); - } - - @Test - public void test() throws Exception { - StateSerializationTest test = new StateSerializationTest(); - test.initializeStateAndTimers(); + private static final StateNamespace NAMESPACE_1 = StateNamespaces.global(); + private static final String KEY_PREFIX = "TEST_"; + + // TODO: This can be replaced with the standard Sum.SumIntererFn once the state no longer needs + // to create a StateTag at the point of restoring state. Currently StateTags are compared strictly + // by type and combiners always use KeyedCombineFnWithContext rather than KeyedCombineFn or CombineFn. + private static CombineWithContext.KeyedCombineFnWithContext SUM_COMBINER = + new CombineWithContext.KeyedCombineFnWithContext() { + @Override + public int[] createAccumulator(Object key, CombineWithContext.Context c) { + return new int[1]; + } + + @Override + public int[] addInput(Object key, int[] accumulator, Integer value, CombineWithContext.Context c) { + accumulator[0] += value; + return accumulator; + } + + @Override + public int[] mergeAccumulators(Object key, Iterable accumulators, CombineWithContext.Context c) { + int[] r = new int[1]; + for (int[] a : accumulators) { + r[0] += a[0]; + } + return r; + } + + @Override + public Integer extractOutput(Object key, int[] accumulator, CombineWithContext.Context c) { + return accumulator[0]; + } + }; + + private static Coder INT_ACCUM_CODER = DelegateCoder.of( + VarIntCoder.of(), + new DelegateCoder.CodingFunction() { + @Override + public Integer apply(int[] accumulator) { + return accumulator[0]; + } + }, + new DelegateCoder.CodingFunction() { + @Override + public int[] apply(Integer value) { + int[] a = new int[1]; + a[0] = value; + return a; + } + }); + + private static final StateTag> STRING_VALUE_ADDR = + StateTags.value("stringValue", StringUtf8Coder.of()); + private static final StateTag> INT_VALUE_ADDR = + StateTags.value("stringValue", VarIntCoder.of()); + private static final StateTag> SUM_INTEGER_ADDR = + StateTags.keyedCombiningValueWithContext("sumInteger", INT_ACCUM_CODER, SUM_COMBINER); + private static final StateTag> STRING_BAG_ADDR = + StateTags.bag("stringBag", StringUtf8Coder.of()); + private static final StateTag> WATERMARK_BAG_ADDR = + StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEarliestInputTimestamp()); + + private Map> statePerKey = new HashMap<>(); + + private Map> activeTimers = new HashMap<>(); + + private void initializeStateAndTimers() throws CannotProvideCoderException { + for (int i = 0; i < 10; i++) { + String key = KEY_PREFIX + i; + + FlinkStateInternals state = initializeStateForKey(key); + Set timers = new HashSet<>(); + for (int j = 0; j < 5; j++) { + TimerInternals.TimerData timer = TimerInternals + .TimerData.of(NAMESPACE_1, + new Instant(1000 + i + j), TimeDomain.values()[j % 3]); + timers.add(timer); + } + + statePerKey.put(key, state); + activeTimers.put(key, timers); + } + } + + private FlinkStateInternals initializeStateForKey(String key) throws CannotProvideCoderException { + FlinkStateInternals state = createState(key); + + ValueState value = state.state(NAMESPACE_1, STRING_VALUE_ADDR); + value.write("test"); + + ValueState value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR); + value2.write(4); + value2.write(5); + + AccumulatorCombiningState combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR); + combiningValue.add(1); + combiningValue.add(2); + + WatermarkHoldState watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + watermark.add(new Instant(1000)); + + BagState bag = state.state(NAMESPACE_1, STRING_BAG_ADDR); + bag.add("v1"); + bag.add("v2"); + bag.add("v3"); + bag.add("v4"); + return state; + } + + private boolean restoreAndTestState(DataInputView in) throws Exception { + StateCheckpointReader reader = new StateCheckpointReader(in); + final ClassLoader userClassloader = this.getClass().getClassLoader(); + Coder windowCoder = IntervalWindow.getCoder(); + Coder keyCoder = StringUtf8Coder.of(); + + boolean comparisonRes = true; + + for (String key : statePerKey.keySet()) { + comparisonRes &= checkStateForKey(key); + } + + // restore the timers + Map> restoredTimersPerKey = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder); + if (activeTimers.size() != restoredTimersPerKey.size()) { + return false; + } + + for (String key : statePerKey.keySet()) { + Set originalTimers = activeTimers.get(key); + Set restoredTimers = restoredTimersPerKey.get(key); + comparisonRes &= checkTimersForKey(originalTimers, restoredTimers); + } + + // restore the state + Map> restoredPerKeyState = + StateCheckpointUtils.decodeState(reader, OutputTimeFns.outputAtEarliestInputTimestamp(), keyCoder, windowCoder, userClassloader); + if (restoredPerKeyState.size() != statePerKey.size()) { + return false; + } + + for (String key : statePerKey.keySet()) { + FlinkStateInternals originalState = statePerKey.get(key); + FlinkStateInternals restoredState = restoredPerKeyState.get(key); + comparisonRes &= checkStateForKey(originalState, restoredState); + } + return comparisonRes; + } + + private boolean checkStateForKey(String key) throws CannotProvideCoderException { + FlinkStateInternals state = statePerKey.get(key); + + ValueState value = state.state(NAMESPACE_1, STRING_VALUE_ADDR); + boolean comp = value.read().equals("test"); + + ValueState value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR); + comp &= value2.read().equals(5); + + AccumulatorCombiningState combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR); + comp &= combiningValue.read().equals(3); + + WatermarkHoldState watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + comp &= watermark.read().equals(new Instant(1000)); + + BagState bag = state.state(NAMESPACE_1, STRING_BAG_ADDR); + Iterator it = bag.read().iterator(); + int i = 0; + while (it.hasNext()) { + comp &= it.next().equals("v" + (++i)); + } + return comp; + } + + private void storeState(AbstractStateBackend.CheckpointStateOutputView out) throws Exception { + StateCheckpointWriter checkpointBuilder = StateCheckpointWriter.create(out); + Coder keyCoder = StringUtf8Coder.of(); + + // checkpoint the timers + StateCheckpointUtils.encodeTimers(activeTimers, checkpointBuilder, keyCoder); + + // checkpoint the state + StateCheckpointUtils.encodeState(statePerKey, checkpointBuilder, keyCoder); + } + + private boolean checkTimersForKey(Set originalTimers, Set restoredTimers) { + boolean comp = true; + if (restoredTimers == null) { + return false; + } + + if (originalTimers.size() != restoredTimers.size()) { + return false; + } + + for (TimerInternals.TimerData timer : originalTimers) { + comp &= restoredTimers.contains(timer); + } + return comp; + } + + private boolean checkStateForKey(FlinkStateInternals originalState, FlinkStateInternals restoredState) throws CannotProvideCoderException { + if (restoredState == null) { + return false; + } + + ValueState orValue = originalState.state(NAMESPACE_1, STRING_VALUE_ADDR); + ValueState resValue = restoredState.state(NAMESPACE_1, STRING_VALUE_ADDR); + boolean comp = orValue.read().equals(resValue.read()); + + ValueState orIntValue = originalState.state(NAMESPACE_1, INT_VALUE_ADDR); + ValueState resIntValue = restoredState.state(NAMESPACE_1, INT_VALUE_ADDR); + comp &= orIntValue.read().equals(resIntValue.read()); + + AccumulatorCombiningState combOrValue = originalState.state(NAMESPACE_1, SUM_INTEGER_ADDR); + AccumulatorCombiningState combResValue = restoredState.state(NAMESPACE_1, SUM_INTEGER_ADDR); + comp &= combOrValue.read().equals(combResValue.read()); + + WatermarkHoldState orWatermark = originalState.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + WatermarkHoldState resWatermark = restoredState.state(NAMESPACE_1, WATERMARK_BAG_ADDR); + comp &= orWatermark.read().equals(resWatermark.read()); + + BagState orBag = originalState.state(NAMESPACE_1, STRING_BAG_ADDR); + BagState resBag = restoredState.state(NAMESPACE_1, STRING_BAG_ADDR); + + Iterator orIt = orBag.read().iterator(); + Iterator resIt = resBag.read().iterator(); + + while (orIt.hasNext() && resIt.hasNext()) { + comp &= orIt.next().equals(resIt.next()); + } + + return !((orIt.hasNext() && !resIt.hasNext()) || (!orIt.hasNext() && resIt.hasNext())) && comp; + } + + private FlinkStateInternals createState(String key) throws CannotProvideCoderException { + return new FlinkStateInternals<>( + key, + StringUtf8Coder.of(), + IntervalWindow.getCoder(), + OutputTimeFns.outputAtEarliestInputTimestamp()); + } + + @Test + public void test() throws Exception { + StateSerializationTest test = new StateSerializationTest(); + test.initializeStateAndTimers(); - MemoryStateBackend.MemoryCheckpointOutputStream memBackend = new MemoryStateBackend.MemoryCheckpointOutputStream(32048); - AbstractStateBackend.CheckpointStateOutputView out = new AbstractStateBackend.CheckpointStateOutputView(memBackend); - - test.storeState(out); + MemoryStateBackend.MemoryCheckpointOutputStream memBackend = new MemoryStateBackend.MemoryCheckpointOutputStream(32048); + AbstractStateBackend.CheckpointStateOutputView out = new AbstractStateBackend.CheckpointStateOutputView(memBackend); + + test.storeState(out); - byte[] contents = memBackend.closeAndGetBytes(); - DataInputView in = new DataInputDeserializer(contents, 0, contents.length); + byte[] contents = memBackend.closeAndGetBytes(); + DataInputView in = new DataInputDeserializer(contents, 0, contents.length); - assertEquals(test.restoreAndTestState(in), true); - } + assertEquals(test.restoreAndTestState(in), true); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java index ad5b53a58ebe..90073c18a63b 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java @@ -40,93 +40,93 @@ * Session window test */ public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable { - protected String resultPath; - - public TopWikipediaSessionsITCase(){ - } - - static final String[] EXPECTED_RESULT = new String[] { - "user: user1 value:3", - "user: user1 value:1", - "user: user2 value:4", - "user: user2 value:6", - "user: user3 value:7", - "user: user3 value:2" - }; - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); - } - - @Override - protected void testProgram() throws Exception { - - Pipeline p = FlinkTestPipeline.createForStreaming(); - - Long now = (System.currentTimeMillis() + 10000) / 1000; - - PCollection> output = - p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set - ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set - ("contributor_username", "user3"), new TableRow().set("timestamp", now).set - ("contributor_username", "user2"), new TableRow().set("timestamp", now).set - ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set - ("contributor_username", "user1"), new TableRow().set("timestamp", now).set - ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set - ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set - ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set - ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set - ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set - ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set - ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set - ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set - ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set - ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set - ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set - ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set - ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set - ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set - ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set - ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set - ("contributor_username", "user2"), new TableRow().set("timestamp", now) - .set("contributor_username", "user3")))) - - - - .apply(ParDo.of(new DoFn() { - @Override - public void processElement(ProcessContext c) throws Exception { - TableRow row = c.element(); - long timestamp = (Integer) row.get("timestamp"); - String userName = (String) row.get("contributor_username"); - if (userName != null) { - // Sets the timestamp field to be used in windowing. - c.outputWithTimestamp(userName, new Instant(timestamp * 1000L)); - } - } - })) - - .apply(Window.into(Sessions.withGapDuration(Duration.standardMinutes(1)))) - - .apply(Count.perElement()); - - PCollection format = output.apply(ParDo.of(new DoFn, String>() { - @Override - public void processElement(ProcessContext c) throws Exception { - KV el = c.element(); - String out = "user: " + el.getKey() + " value:" + el.getValue(); - c.output(out); - } - })); - - format.apply(TextIO.Write.to(resultPath)); - - p.run(); - } + protected String resultPath; + + public TopWikipediaSessionsITCase(){ + } + + static final String[] EXPECTED_RESULT = new String[] { + "user: user1 value:3", + "user: user1 value:1", + "user: user2 value:4", + "user: user2 value:6", + "user: user3 value:7", + "user: user3 value:2" + }; + + @Override + protected void preSubmit() throws Exception { + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); + } + + @Override + protected void testProgram() throws Exception { + + Pipeline p = FlinkTestPipeline.createForStreaming(); + + Long now = (System.currentTimeMillis() + 10000) / 1000; + + PCollection> output = + p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set + ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now).set + ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set + ("contributor_username", "user1"), new TableRow().set("timestamp", now).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set + ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set + ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set + ("contributor_username", "user2"), new TableRow().set("timestamp", now) + .set("contributor_username", "user3")))) + + + + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + TableRow row = c.element(); + long timestamp = (Integer) row.get("timestamp"); + String userName = (String) row.get("contributor_username"); + if (userName != null) { + // Sets the timestamp field to be used in windowing. + c.outputWithTimestamp(userName, new Instant(timestamp * 1000L)); + } + } + })) + + .apply(Window.into(Sessions.withGapDuration(Duration.standardMinutes(1)))) + + .apply(Count.perElement()); + + PCollection format = output.apply(ParDo.of(new DoFn, String>() { + @Override + public void processElement(ProcessContext c) throws Exception { + KV el = c.element(); + String out = "user: " + el.getKey() + " value:" + el.getValue(); + c.output(out); + } + })); + + format.apply(TextIO.Write.to(resultPath)); + + p.run(); + } } diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java index aa5623d6c39d..b1ccee435daf 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java @@ -38,121 +38,121 @@ */ public class JoinExamples { - // A 1000-row sample of the GDELT data here: gdelt-bq:full.events. - private static final String GDELT_EVENTS_TABLE = - "clouddataflow-readonly:samples.gdelt_sample"; - // A table that maps country codes to country names. - private static final String COUNTRY_CODES = - "gdelt-bq:full.crosswalk_geocountrycodetohuman"; - - /** - * Join two collections, using country code as the key. - */ - public static PCollection joinEvents(PCollection eventsTable, - PCollection countryCodes) throws Exception { - - final TupleTag eventInfoTag = new TupleTag<>(); - final TupleTag countryInfoTag = new TupleTag<>(); - - // transform both input collections to tuple collections, where the keys are country - // codes in both cases. - PCollection> eventInfo = eventsTable.apply( - ParDo.of(new ExtractEventDataFn())); - PCollection> countryInfo = countryCodes.apply( - ParDo.of(new ExtractCountryInfoFn())); - - // country code 'key' -> CGBKR (, ) - PCollection> kvpCollection = KeyedPCollectionTuple - .of(eventInfoTag, eventInfo) - .and(countryInfoTag, countryInfo) - .apply(CoGroupByKey.create()); - - // Process the CoGbkResult elements generated by the CoGroupByKey transform. - // country code 'key' -> string of , - PCollection> finalResultCollection = - kvpCollection.apply(ParDo.of(new DoFn, KV>() { - @Override - public void processElement(ProcessContext c) { - KV e = c.element(); - CoGbkResult val = e.getValue(); - String countryCode = e.getKey(); - String countryName; - countryName = e.getValue().getOnly(countryInfoTag, "Kostas"); - for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) { - // Generate a string that combines information from both collection values - c.output(KV.of(countryCode, "Country name: " + countryName - + ", Event info: " + eventInfo)); - } - } - })); - - // write to GCS - return finalResultCollection - .apply(ParDo.of(new DoFn, String>() { - @Override - public void processElement(ProcessContext c) { - String outputstring = "Country code: " + c.element().getKey() - + ", " + c.element().getValue(); - c.output(outputstring); - } - })); - } - - /** - * Examines each row (event) in the input table. Output a KV with the key the country - * code of the event, and the value a string encoding event information. - */ - static class ExtractEventDataFn extends DoFn> { - @Override - public void processElement(ProcessContext c) { - TableRow row = c.element(); - String countryCode = (String) row.get("ActionGeo_CountryCode"); - String sqlDate = (String) row.get("SQLDATE"); - String actor1Name = (String) row.get("Actor1Name"); - String sourceUrl = (String) row.get("SOURCEURL"); - String eventInfo = "Date: " + sqlDate + ", Actor1: " + actor1Name + ", url: " + sourceUrl; - c.output(KV.of(countryCode, eventInfo)); - } - } - - - /** - * Examines each row (country info) in the input table. Output a KV with the key the country - * code, and the value the country name. - */ - static class ExtractCountryInfoFn extends DoFn> { - @Override - public void processElement(ProcessContext c) { - TableRow row = c.element(); - String countryCode = (String) row.get("FIPSCC"); - String countryName = (String) row.get("HumanName"); - c.output(KV.of(countryCode, countryName)); - } - } - - - /** - * Options supported by {@link JoinExamples}. - *

    - * Inherits standard configuration options. - */ - private interface Options extends PipelineOptions { - @Description("Path of the file to write to") - @Validation.Required - String getOutput(); - void setOutput(String value); - } - - public static void main(String[] args) throws Exception { - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - Pipeline p = Pipeline.create(options); - // the following two 'applys' create multiple inputs to our pipeline, one for each - // of our two input sources. - PCollection eventsTable = p.apply(BigQueryIO.Read.from(GDELT_EVENTS_TABLE)); - PCollection countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES)); - PCollection formattedResults = joinEvents(eventsTable, countryCodes); - formattedResults.apply(TextIO.Write.to(options.getOutput())); - p.run(); - } + // A 1000-row sample of the GDELT data here: gdelt-bq:full.events. + private static final String GDELT_EVENTS_TABLE = + "clouddataflow-readonly:samples.gdelt_sample"; + // A table that maps country codes to country names. + private static final String COUNTRY_CODES = + "gdelt-bq:full.crosswalk_geocountrycodetohuman"; + + /** + * Join two collections, using country code as the key. + */ + public static PCollection joinEvents(PCollection eventsTable, + PCollection countryCodes) throws Exception { + + final TupleTag eventInfoTag = new TupleTag<>(); + final TupleTag countryInfoTag = new TupleTag<>(); + + // transform both input collections to tuple collections, where the keys are country + // codes in both cases. + PCollection> eventInfo = eventsTable.apply( + ParDo.of(new ExtractEventDataFn())); + PCollection> countryInfo = countryCodes.apply( + ParDo.of(new ExtractCountryInfoFn())); + + // country code 'key' -> CGBKR (, ) + PCollection> kvpCollection = KeyedPCollectionTuple + .of(eventInfoTag, eventInfo) + .and(countryInfoTag, countryInfo) + .apply(CoGroupByKey.create()); + + // Process the CoGbkResult elements generated by the CoGroupByKey transform. + // country code 'key' -> string of , + PCollection> finalResultCollection = + kvpCollection.apply(ParDo.of(new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) { + KV e = c.element(); + CoGbkResult val = e.getValue(); + String countryCode = e.getKey(); + String countryName; + countryName = e.getValue().getOnly(countryInfoTag, "Kostas"); + for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) { + // Generate a string that combines information from both collection values + c.output(KV.of(countryCode, "Country name: " + countryName + + ", Event info: " + eventInfo)); + } + } + })); + + // write to GCS + return finalResultCollection + .apply(ParDo.of(new DoFn, String>() { + @Override + public void processElement(ProcessContext c) { + String outputstring = "Country code: " + c.element().getKey() + + ", " + c.element().getValue(); + c.output(outputstring); + } + })); + } + + /** + * Examines each row (event) in the input table. Output a KV with the key the country + * code of the event, and the value a string encoding event information. + */ + static class ExtractEventDataFn extends DoFn> { + @Override + public void processElement(ProcessContext c) { + TableRow row = c.element(); + String countryCode = (String) row.get("ActionGeo_CountryCode"); + String sqlDate = (String) row.get("SQLDATE"); + String actor1Name = (String) row.get("Actor1Name"); + String sourceUrl = (String) row.get("SOURCEURL"); + String eventInfo = "Date: " + sqlDate + ", Actor1: " + actor1Name + ", url: " + sourceUrl; + c.output(KV.of(countryCode, eventInfo)); + } + } + + + /** + * Examines each row (country info) in the input table. Output a KV with the key the country + * code, and the value the country name. + */ + static class ExtractCountryInfoFn extends DoFn> { + @Override + public void processElement(ProcessContext c) { + TableRow row = c.element(); + String countryCode = (String) row.get("FIPSCC"); + String countryName = (String) row.get("HumanName"); + c.output(KV.of(countryCode, countryName)); + } + } + + + /** + * Options supported by {@link JoinExamples}. + *

    + * Inherits standard configuration options. + */ + private interface Options extends PipelineOptions { + @Description("Path of the file to write to") + @Validation.Required + String getOutput(); + void setOutput(String value); + } + + public static void main(String[] args) throws Exception { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + Pipeline p = Pipeline.create(options); + // the following two 'applys' create multiple inputs to our pipeline, one for each + // of our two input sources. + PCollection eventsTable = p.apply(BigQueryIO.Read.from(GDELT_EVENTS_TABLE)); + PCollection countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES)); + PCollection formattedResults = joinEvents(eventsTable, countryCodes); + formattedResults.apply(TextIO.Write.to(options.getOutput())); + p.run(); + } } From 6c2991097865add504ac02ef282040a4b01f4294 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 2 Mar 2016 23:12:41 +0100 Subject: [PATCH 144/149] [flink] change package namespace --- .../FlinkPipelineExecutionEnvironment.java | 8 ++--- .../flink/dataflow/FlinkPipelineOptions.java | 2 +- .../flink/dataflow/FlinkPipelineRunner.java | 2 +- .../flink/dataflow/FlinkRunnerResult.java | 2 +- .../flink/dataflow/examples/TFIDF.java | 6 ++-- .../flink/dataflow/examples/WordCount.java | 6 ++-- .../examples/streaming/AutoComplete.java | 6 ++-- .../examples/streaming/JoinExamples.java | 6 ++-- .../KafkaWindowedWordCountExample.java | 6 ++-- .../examples/streaming/WindowedWordCount.java | 8 ++--- .../flink/dataflow/io/ConsoleIO.java | 4 +-- .../FlinkBatchPipelineTranslator.java | 2 +- .../FlinkBatchTransformTranslators.java | 32 +++++++++---------- .../FlinkBatchTranslationContext.java | 6 ++-- .../translation/FlinkPipelineTranslator.java | 2 +- .../FlinkStreamingPipelineTranslator.java | 2 +- .../FlinkStreamingTransformTranslators.java | 16 +++++----- .../FlinkStreamingTranslationContext.java | 2 +- .../FlinkCoGroupKeyedListAggregator.java | 2 +- .../functions/FlinkCreateFunction.java | 4 +-- .../functions/FlinkDoFnFunction.java | 4 +-- .../FlinkKeyedListAggregationFunction.java | 2 +- .../FlinkMultiOutputDoFnFunction.java | 4 +-- .../FlinkMultiOutputPruningFunction.java | 2 +- .../functions/FlinkPartialReduceFunction.java | 2 +- .../functions/FlinkReduceFunction.java | 2 +- .../translation/functions/UnionCoder.java | 2 +- .../translation/types/CoderComparator.java | 2 +- .../types/CoderTypeInformation.java | 2 +- .../types/CoderTypeSerializer.java | 6 ++-- .../InspectableByteArrayOutputStream.java | 2 +- .../translation/types/KvCoderComperator.java | 4 +-- .../types/KvCoderTypeInformation.java | 2 +- .../types/VoidCoderTypeSerializer.java | 2 +- .../wrappers/CombineFnAggregatorWrapper.java | 2 +- .../wrappers/DataInputViewWrapper.java | 2 +- .../wrappers/DataOutputViewWrapper.java | 2 +- .../SerializableFnAggregatorWrapper.java | 2 +- .../wrappers/SinkOutputFormat.java | 2 +- .../wrappers/SourceInputFormat.java | 2 +- .../wrappers/SourceInputSplit.java | 4 +-- .../streaming/FlinkAbstractParDoWrapper.java | 4 +-- .../FlinkGroupAlsoByWindowWrapper.java | 8 ++--- .../streaming/FlinkGroupByKeyWrapper.java | 6 ++-- .../FlinkParDoBoundMultiWrapper.java | 2 +- .../streaming/FlinkParDoBoundWrapper.java | 2 +- .../io/FlinkStreamingCreateFunction.java | 4 +-- .../streaming/io/UnboundedFlinkSource.java | 4 +-- .../streaming/io/UnboundedSocketSource.java | 2 +- .../streaming/io/UnboundedSourceWrapper.java | 2 +- .../state/AbstractFlinkTimerInternals.java | 4 +-- .../streaming/state/FlinkStateInternals.java | 2 +- .../state/StateCheckpointReader.java | 4 +-- .../streaming/state/StateCheckpointUtils.java | 4 +-- .../state/StateCheckpointWriter.java | 4 +-- .../wrappers/streaming/state/StateType.java | 2 +- .../flink/dataflow/AvroITCase.java | 2 +- .../flink/dataflow/FlattenizeITCase.java | 2 +- .../flink/dataflow/FlinkTestPipeline.java | 4 +-- .../flink/dataflow/JoinExamplesITCase.java | 4 +-- .../flink/dataflow/MaybeEmptyTestITCase.java | 2 +- .../dataflow/ParDoMultiOutputITCase.java | 2 +- .../flink/dataflow/ReadSourceITCase.java | 2 +- .../dataflow/RemoveDuplicatesEmptyITCase.java | 2 +- .../dataflow/RemoveDuplicatesITCase.java | 2 +- .../flink/dataflow/SideInputITCase.java | 2 +- .../flink/dataflow/TfIdfITCase.java | 4 +-- .../flink/dataflow/WordCountITCase.java | 4 +-- .../flink/dataflow/WordCountJoin2ITCase.java | 2 +- .../flink/dataflow/WordCountJoin3ITCase.java | 2 +- .../flink/dataflow/WriteSinkITCase.java | 2 +- .../streaming/GroupAlsoByWindowTest.java | 6 ++-- .../streaming/GroupByNullKeyTest.java | 4 +-- .../streaming/StateSerializationTest.java | 10 +++--- .../streaming/TopWikipediaSessionsITCase.java | 4 +-- .../flink/dataflow/util/JoinExamples.java | 2 +- 76 files changed, 148 insertions(+), 148 deletions(-) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java index c2139c68b47a..02a49b9b0856 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java @@ -13,11 +13,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; -import com.dataartisans.flink.dataflow.translation.FlinkPipelineTranslator; -import com.dataartisans.flink.dataflow.translation.FlinkBatchPipelineTranslator; -import com.dataartisans.flink.dataflow.translation.FlinkStreamingPipelineTranslator; +import org.apache.beam.runners.flink.translation.FlinkPipelineTranslator; +import org.apache.beam.runners.flink.translation.FlinkBatchPipelineTranslator; +import org.apache.beam.runners.flink.translation.FlinkStreamingPipelineTranslator; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.common.base.Preconditions; import org.apache.flink.api.common.JobExecutionResult; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java index fabbfadb980b..bf833534263d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.fasterxml.jackson.annotation.JsonIgnore; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java index 742a31643239..3c33d2088889 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.options.PipelineOptions; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java index dfbaf66f41ad..c2329a62482b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.PipelineResult; import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java index 8accae7dbe16..ab23b926e2a7 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java @@ -14,10 +14,10 @@ * the License. */ -package com.dataartisans.flink.dataflow.examples; +package org.apache.beam.runners.flink.examples; -import com.dataartisans.flink.dataflow.FlinkPipelineOptions; -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +import org.apache.beam.runners.flink.FlinkPipelineOptions; +import org.apache.beam.runners.flink.FlinkPipelineRunner; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java index 4f721b4c10ae..ba4630164dbf 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java @@ -13,10 +13,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.examples; +package org.apache.beam.runners.flink.examples; -import com.dataartisans.flink.dataflow.FlinkPipelineOptions; -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +import org.apache.beam.runners.flink.FlinkPipelineOptions; +import org.apache.beam.runners.flink.FlinkPipelineRunner; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.options.Default; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java index 493fb25c3b80..816812215feb 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java @@ -14,10 +14,10 @@ * the License. */ -package com.dataartisans.flink.dataflow.examples.streaming; +package org.apache.beam.runners.flink.examples.streaming; -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSocketSource; +import org.apache.beam.runners.flink.FlinkPipelineRunner; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.coders.DefaultCoder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java index a6e1e37fbe04..3a8bdb0078d6 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java @@ -14,10 +14,10 @@ * the License. */ -package com.dataartisans.flink.dataflow.examples.streaming; +package org.apache.beam.runners.flink.examples.streaming; -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSocketSource; +import org.apache.beam.runners.flink.FlinkPipelineRunner; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java index b97c35cfb556..fa0c8e9b7bea 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java @@ -13,10 +13,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.examples.streaming; +package org.apache.beam.runners.flink.examples.streaming; -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedFlinkSource; +import org.apache.beam.runners.flink.FlinkPipelineRunner; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java index 753cbc3ec71d..6af044d18d5f 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java @@ -13,10 +13,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.examples.streaming; +package org.apache.beam.runners.flink.examples.streaming; -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSocketSource; +import org.apache.beam.runners.flink.FlinkPipelineRunner; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.*; import com.google.cloud.dataflow.sdk.options.Default; @@ -80,7 +80,7 @@ public void processElement(ProcessContext c) { } } - public interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options { + public interface StreamingWordCountOptions extends org.apache.beam.runners.flink.examples.WordCount.Options { @Description("Sliding window duration, in seconds") @Default.Long(WINDOW_SIZE) Long getWindowSize(); diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java index 3f3492ccf610..cd25ba3d5735 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.io; +package org.apache.beam.runners.flink.io; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -24,7 +24,7 @@ * to standard output. * * This is Flink-specific and will only work when executed using the - * {@link com.dataartisans.flink.dataflow.FlinkPipelineRunner}. + * {@link org.apache.beam.runners.flink.FlinkPipelineRunner}. */ public class ConsoleIO { diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java index 82b7e970cd40..520142357f28 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation; +package org.apache.beam.runners.flink.translation; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java index 6a8409c71ddb..f33e4f527476 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java @@ -15,22 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation; - -import com.dataartisans.flink.dataflow.io.ConsoleIO; -import com.dataartisans.flink.dataflow.translation.functions.FlinkCoGroupKeyedListAggregator; -import com.dataartisans.flink.dataflow.translation.functions.FlinkCreateFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkDoFnFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkKeyedListAggregationFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkMultiOutputDoFnFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkMultiOutputPruningFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkPartialReduceFunction; -import com.dataartisans.flink.dataflow.translation.functions.FlinkReduceFunction; -import com.dataartisans.flink.dataflow.translation.functions.UnionCoder; -import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; -import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation; -import com.dataartisans.flink.dataflow.translation.wrappers.SinkOutputFormat; -import com.dataartisans.flink.dataflow.translation.wrappers.SourceInputFormat; +package org.apache.beam.runners.flink.translation; + +import org.apache.beam.runners.flink.io.ConsoleIO; +import org.apache.beam.runners.flink.translation.functions.FlinkCoGroupKeyedListAggregator; +import org.apache.beam.runners.flink.translation.functions.FlinkCreateFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkKeyedListAggregationFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputDoFnFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputPruningFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkPartialReduceFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkReduceFunction; +import org.apache.beam.runners.flink.translation.functions.UnionCoder; +import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; +import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation; +import org.apache.beam.runners.flink.translation.wrappers.SinkOutputFormat; +import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat; import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException; import com.google.cloud.dataflow.sdk.coders.Coder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java index ec1d6c841f81..fd99833f056b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation; +package org.apache.beam.runners.flink.translation; -import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; -import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation; +import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; +import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java index a6a333bab9a6..efe217f9c1c8 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation; +package org.apache.beam.runners.flink.translation; import com.google.cloud.dataflow.sdk.Pipeline; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java index 897303d5e0ee..21a8133dae98 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation; +package org.apache.beam.runners.flink.translation; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java index 9fd33be1a7ce..10838485d9c2 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java @@ -14,14 +14,14 @@ * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation; - -import com.dataartisans.flink.dataflow.translation.functions.UnionCoder; -import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.*; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.FlinkStreamingCreateFunction; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedFlinkSource; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSourceWrapper; +package org.apache.beam.runners.flink.translation; + +import org.apache.beam.runners.flink.translation.functions.UnionCoder; +import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; +import org.apache.beam.runners.flink.translation.wrappers.streaming.*; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.FlinkStreamingCreateFunction; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource; +import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper; import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java index 3586d0c871c7..83b19eee0415 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation; +package org.apache.beam.runners.flink.translation; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java index 5897473fc699..0e855addf9d3 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.functions; +package org.apache.beam.runners.flink.translation.functions; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult; import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java index 03f2b06b4cc3..32efe4120226 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.functions; +package org.apache.beam.runners.flink.translation.functions; -import com.dataartisans.flink.dataflow.translation.types.VoidCoderTypeSerializer; +import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; import com.google.cloud.dataflow.sdk.coders.Coder; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java index 53ff1cf4e5ff..9e8b7cd2548d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.functions; +package org.apache.beam.runners.flink.translation.functions; -import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper; +import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java index 011697293eef..56fa88da649a 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.functions; +package org.apache.beam.runners.flink.translation.functions; import com.google.cloud.dataflow.sdk.values.KV; import org.apache.flink.api.common.functions.GroupReduceFunction; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java index 9e516389e95d..bd8a968f0d72 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.functions; +package org.apache.beam.runners.flink.translation.functions; -import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper; +import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java index e883d42bc97b..3e1cb65ed4ee 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.functions; +package org.apache.beam.runners.flink.translation.functions; import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue; import org.apache.flink.api.common.functions.FlatMapFunction; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java index 1ff9aff88ab6..1ff06ba81b18 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.functions; +package org.apache.beam.runners.flink.translation.functions; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.values.KV; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java index 94676a2896dc..580ac01b928b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.functions; +package org.apache.beam.runners.flink.translation.functions; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.values.KV; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java index 3cc5c2440570..05f441551a7b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java @@ -14,7 +14,7 @@ * the License. */ -package com.dataartisans.flink.dataflow.translation.functions; +package org.apache.beam.runners.flink.translation.functions; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java index b402f7c4f30d..ecfb95d0d535 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.types; +package org.apache.beam.runners.flink.translation.types; import com.google.cloud.dataflow.sdk.coders.Coder; import org.apache.flink.api.common.typeutils.TypeComparator; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java index ae4309ea02dd..8880b4824339 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.types; +package org.apache.beam.runners.flink.translation.types; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java index 6ed661c33ab0..481ee3124e14 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.types; +package org.apache.beam.runners.flink.translation.types; -import com.dataartisans.flink.dataflow.translation.wrappers.DataInputViewWrapper; -import com.dataartisans.flink.dataflow.translation.wrappers.DataOutputViewWrapper; +import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; +import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderException; import org.apache.flink.api.common.typeutils.TypeSerializer; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java index be6eadd160ff..619fa55b9251 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.types; +package org.apache.beam.runners.flink.translation.types; import java.io.ByteArrayOutputStream; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java index ba09ea91f361..4599c6ac4c0d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.types; +package org.apache.beam.runners.flink.translation.types; -import com.dataartisans.flink.dataflow.translation.wrappers.DataInputViewWrapper; +import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.values.KV; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java index be119182bb3a..7a0d9993209b 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.types; +package org.apache.beam.runners.flink.translation.types; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.values.KV; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java index 190d898cfe5d..c7b6ea286077 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.types; +package org.apache.beam.runners.flink.translation.types; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java index 8f6d67cbae5c..815765c4ac67 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers; +package org.apache.beam.runners.flink.translation.wrappers; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java index 3c9693949be2..b56a90ef31fd 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers; +package org.apache.beam.runners.flink.translation.wrappers; import org.apache.flink.core.memory.DataInputView; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java index a222cdd93392..513d7f87986c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers; +package org.apache.beam.runners.flink.translation.wrappers; import org.apache.flink.core.memory.DataOutputView; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java index c193a4dedca8..0d03f9f2307c 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers; +package org.apache.beam.runners.flink.translation.wrappers; import com.google.cloud.dataflow.sdk.transforms.Aggregator; import com.google.cloud.dataflow.sdk.transforms.Combine; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java index 3f28c1630b77..d0423b923cad 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers; +package org.apache.beam.runners.flink.translation.wrappers; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.cloud.dataflow.sdk.io.Sink; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java index 598161842e5c..2d62416ab92a 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers; +package org.apache.beam.runners.flink.translation.wrappers; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.cloud.dataflow.sdk.coders.Coder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java index 86fdadafa8fa..1b45ad7afdb0 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers; +package org.apache.beam.runners.flink.translation.wrappers; import com.google.cloud.dataflow.sdk.io.Source; import org.apache.flink.core.io.InputSplit; /** * {@link org.apache.flink.core.io.InputSplit} for - * {@link com.dataartisans.flink.dataflow.translation.wrappers.SourceInputFormat}. We pass + * {@link org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat}. We pass * the sharded Source around in the input split because Sources simply split up into several * Sources for sharding. This is different to how Flink creates a separate InputSplit from * an InputFormat. diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index 7dae0b0443c5..e2ceae6282c9 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming; +package org.apache.beam.runners.flink.translation.wrappers.streaming; -import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper; +import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.common.base.Preconditions; import com.google.cloud.dataflow.sdk.transforms.Aggregator; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java index 55235c9d4725..906a39940c7d 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -13,11 +13,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming; +package org.apache.beam.runners.flink.translation.wrappers.streaming; -import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; -import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.*; +import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; +import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; +import org.apache.beam.runners.flink.translation.wrappers.streaming.state.*; import com.google.cloud.dataflow.sdk.coders.*; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java index d01cf81a12c3..61953a69e5f1 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java @@ -13,10 +13,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming; +package org.apache.beam.runners.flink.translation.wrappers.streaming; -import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation; -import com.dataartisans.flink.dataflow.translation.types.VoidCoderTypeSerializer; +import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; +import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java index 066a55ca1fb2..cdf23f6124ab 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming; +package org.apache.beam.runners.flink.translation.wrappers.streaming; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.DoFn; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java index b3a7090e6bb5..3357cd5f7fdc 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming; +package org.apache.beam.runners.flink.translation.wrappers.streaming; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java index 39770c924ff4..2599e88c51f7 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming.io; +package org.apache.beam.runners.flink.translation.wrappers.streaming.io; -import com.dataartisans.flink.dataflow.translation.types.VoidCoderTypeSerializer; +import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java index 4d6f4e245137..ddbc99390174 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming.io; +package org.apache.beam.runners.flink.translation.wrappers.streaming.io; -import com.dataartisans.flink.dataflow.FlinkPipelineRunner; +import org.apache.beam.runners.flink.FlinkPipelineRunner; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.UnboundedSource; import com.google.cloud.dataflow.sdk.options.PipelineOptions; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java index 0b1a5daa317c..a24964a919cf 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming.io; +package org.apache.beam.runners.flink.translation.wrappers.streaming.io; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index 5a89894a7374..7c1ccdf92a24 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming.io; +package org.apache.beam.runners.flink.translation.wrappers.streaming.io; import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.UnboundedSource; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java index 75c8ac6560c1..7accf09ff621 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; +package org.apache.beam.runners.flink.translation.wrappers.streaming.state; -import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer; +import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.transforms.DoFn; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java index 39fec1403962..84007af09986 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; +package org.apache.beam.runners.flink.translation.wrappers.streaming.state; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java index 753309ef244c..d73ac8c05749 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; +package org.apache.beam.runners.flink.translation.wrappers.streaming.state; -import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer; +import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer; import com.google.protobuf.ByteString; import org.apache.flink.core.memory.DataInputView; import org.joda.time.Instant; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java index 17418293ecfe..055a12ae25c4 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; +package org.apache.beam.runners.flink.translation.wrappers.streaming.state; -import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer; +import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java index 9f602fd71e50..738ce5fb8962 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; +package org.apache.beam.runners.flink.translation.wrappers.streaming.state; -import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer; +import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer; import com.google.protobuf.ByteString; import org.apache.flink.runtime.state.AbstractStateBackend; import org.joda.time.Instant; diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java index 9e2c9f8158a6..8b206006a636 100644 --- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java +++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.translation.wrappers.streaming.state; +package org.apache.beam.runners.flink.translation.wrappers.streaming.state; import java.io.IOException; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java index 327297592679..eaa597988692 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.AvroCoder; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java index e65e4977473e..79eb16329383 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java index 578e0e142baa..2dcebdef5520 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.PipelineResult; @@ -22,7 +22,7 @@ /** * {@link com.google.cloud.dataflow.sdk.Pipeline} for testing Dataflow programs on the - * {@link com.dataartisans.flink.dataflow.FlinkPipelineRunner}. + * {@link org.apache.beam.runners.flink.FlinkPipelineRunner}. */ public class FlinkTestPipeline extends Pipeline { diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java index 28861eaa27e4..11b6ce40171c 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; -import com.dataartisans.flink.dataflow.util.JoinExamples; +import org.apache.beam.runners.flink.util.JoinExamples; import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java index d1652e7b4ffe..e39b81def51f 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.VoidCoder; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java index d8087d6563ab..08e532330f34 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java index 5a463590f715..72024175e1d4 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java index 615f19420a84..dc82d7d21df2 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java index 8c19f2cb6c50..78b48b5ecc7d 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java index 7c3d6f936859..5cd7d78e577e 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java index 715d0be7704a..ceb0a3f1c053 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; -import com.dataartisans.flink.dataflow.examples.TFIDF; +import org.apache.beam.runners.flink.examples.TFIDF; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java index f1a2454c16a2..c2b6fdd55ead 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; -import com.dataartisans.flink.dataflow.examples.WordCount; +import org.apache.beam.runners.flink.examples.WordCount; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java index 1cac036e1026..d78434b2427c 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java index 4c8b99b3c484..083627943768 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java index a61bf52b2f99..497a5bb8868e 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.dataartisans.flink.dataflow; +package org.apache.beam.runners.flink; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.Coder; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java index fb240f4412fb..27ddc836ee90 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java @@ -13,10 +13,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.streaming; +package org.apache.beam.runners.flink.streaming; -import com.dataartisans.flink.dataflow.FlinkTestPipeline; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.FlinkGroupAlsoByWindowWrapper; +import org.apache.beam.runners.flink.FlinkTestPipeline; +import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkGroupAlsoByWindowWrapper; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java index 52e9e25b442e..80d78b94471f 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.streaming; +package org.apache.beam.runners.flink.streaming; -import com.dataartisans.flink.dataflow.FlinkTestPipeline; +import org.apache.beam.runners.flink.FlinkTestPipeline; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; import com.google.cloud.dataflow.sdk.transforms.Create; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java index d5b10437d778..e6c5ae2fd17d 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java @@ -13,12 +13,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.streaming; +package org.apache.beam.runners.flink.streaming; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.FlinkStateInternals; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointReader; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointUtils; -import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointWriter; +import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals; +import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointReader; +import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointUtils; +import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointWriter; import com.google.cloud.dataflow.sdk.coders.*; import com.google.cloud.dataflow.sdk.transforms.Combine; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java index 90073c18a63b..f0b93a00d205 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java @@ -13,9 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.streaming; +package org.apache.beam.runners.flink.streaming; -import com.dataartisans.flink.dataflow.FlinkTestPipeline; +import org.apache.beam.runners.flink.FlinkTestPipeline; import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.TextIO; diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java index b1ccee435daf..620dace09d1c 100644 --- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java +++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dataartisans.flink.dataflow.util; +package org.apache.beam.runners.flink.util; import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.dataflow.sdk.Pipeline; From 25f44fffc59cf06598250f9b6f5a1f36e0dcb811 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 2 Mar 2016 23:15:22 +0100 Subject: [PATCH 145/149] [flink] adjust directories according to package name --- .../beam/runners/flink}/FlinkPipelineExecutionEnvironment.java | 0 .../apache/beam/runners/flink}/FlinkPipelineOptions.java | 0 .../apache/beam/runners/flink}/FlinkPipelineRunner.java | 0 .../apache/beam/runners/flink}/FlinkRunnerResult.java | 0 .../apache/beam/runners/flink}/examples/TFIDF.java | 0 .../apache/beam/runners/flink}/examples/WordCount.java | 0 .../beam/runners/flink}/examples/streaming/AutoComplete.java | 0 .../beam/runners/flink}/examples/streaming/JoinExamples.java | 0 .../flink}/examples/streaming/KafkaWindowedWordCountExample.java | 0 .../beam/runners/flink}/examples/streaming/WindowedWordCount.java | 0 .../dataflow => org/apache/beam/runners/flink}/io/ConsoleIO.java | 0 .../runners/flink}/translation/FlinkBatchPipelineTranslator.java | 0 .../flink}/translation/FlinkBatchTransformTranslators.java | 0 .../runners/flink}/translation/FlinkBatchTranslationContext.java | 0 .../beam/runners/flink}/translation/FlinkPipelineTranslator.java | 0 .../flink}/translation/FlinkStreamingPipelineTranslator.java | 0 .../flink}/translation/FlinkStreamingTransformTranslators.java | 0 .../flink}/translation/FlinkStreamingTranslationContext.java | 0 .../translation/functions/FlinkCoGroupKeyedListAggregator.java | 0 .../runners/flink}/translation/functions/FlinkCreateFunction.java | 0 .../runners/flink}/translation/functions/FlinkDoFnFunction.java | 0 .../translation/functions/FlinkKeyedListAggregationFunction.java | 0 .../translation/functions/FlinkMultiOutputDoFnFunction.java | 0 .../translation/functions/FlinkMultiOutputPruningFunction.java | 0 .../flink}/translation/functions/FlinkPartialReduceFunction.java | 0 .../runners/flink}/translation/functions/FlinkReduceFunction.java | 0 .../beam/runners/flink}/translation/functions/UnionCoder.java | 0 .../beam/runners/flink}/translation/types/CoderComparator.java | 0 .../runners/flink}/translation/types/CoderTypeInformation.java | 0 .../runners/flink}/translation/types/CoderTypeSerializer.java | 0 .../translation/types/InspectableByteArrayOutputStream.java | 0 .../beam/runners/flink}/translation/types/KvCoderComperator.java | 0 .../runners/flink}/translation/types/KvCoderTypeInformation.java | 0 .../runners/flink}/translation/types/VoidCoderTypeSerializer.java | 0 .../flink}/translation/wrappers/CombineFnAggregatorWrapper.java | 0 .../runners/flink}/translation/wrappers/DataInputViewWrapper.java | 0 .../flink}/translation/wrappers/DataOutputViewWrapper.java | 0 .../translation/wrappers/SerializableFnAggregatorWrapper.java | 0 .../runners/flink}/translation/wrappers/SinkOutputFormat.java | 0 .../runners/flink}/translation/wrappers/SourceInputFormat.java | 0 .../runners/flink}/translation/wrappers/SourceInputSplit.java | 0 .../translation/wrappers/streaming/FlinkAbstractParDoWrapper.java | 0 .../wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java | 0 .../translation/wrappers/streaming/FlinkGroupByKeyWrapper.java | 0 .../wrappers/streaming/FlinkParDoBoundMultiWrapper.java | 0 .../translation/wrappers/streaming/FlinkParDoBoundWrapper.java | 0 .../wrappers/streaming/io/FlinkStreamingCreateFunction.java | 0 .../translation/wrappers/streaming/io/UnboundedFlinkSource.java | 0 .../translation/wrappers/streaming/io/UnboundedSocketSource.java | 0 .../translation/wrappers/streaming/io/UnboundedSourceWrapper.java | 0 .../wrappers/streaming/state/AbstractFlinkTimerInternals.java | 0 .../translation/wrappers/streaming/state/FlinkStateInternals.java | 0 .../wrappers/streaming/state/StateCheckpointReader.java | 0 .../wrappers/streaming/state/StateCheckpointUtils.java | 0 .../wrappers/streaming/state/StateCheckpointWriter.java | 0 .../flink}/translation/wrappers/streaming/state/StateType.java | 0 .../dataflow => org/apache/beam/runners/flink}/AvroITCase.java | 0 .../apache/beam/runners/flink}/FlattenizeITCase.java | 0 .../apache/beam/runners/flink}/FlinkTestPipeline.java | 0 .../apache/beam/runners/flink}/JoinExamplesITCase.java | 0 .../apache/beam/runners/flink}/MaybeEmptyTestITCase.java | 0 .../apache/beam/runners/flink}/ParDoMultiOutputITCase.java | 0 .../apache/beam/runners/flink}/ReadSourceITCase.java | 0 .../apache/beam/runners/flink}/RemoveDuplicatesEmptyITCase.java | 0 .../apache/beam/runners/flink}/RemoveDuplicatesITCase.java | 0 .../apache/beam/runners/flink}/SideInputITCase.java | 0 .../dataflow => org/apache/beam/runners/flink}/TfIdfITCase.java | 0 .../apache/beam/runners/flink}/WordCountITCase.java | 0 .../apache/beam/runners/flink}/WordCountJoin2ITCase.java | 0 .../apache/beam/runners/flink}/WordCountJoin3ITCase.java | 0 .../apache/beam/runners/flink}/WriteSinkITCase.java | 0 .../beam/runners/flink}/streaming/GroupAlsoByWindowTest.java | 0 .../apache/beam/runners/flink}/streaming/GroupByNullKeyTest.java | 0 .../beam/runners/flink}/streaming/StateSerializationTest.java | 0 .../beam/runners/flink}/streaming/TopWikipediaSessionsITCase.java | 0 .../apache/beam/runners/flink}/util/JoinExamples.java | 0 76 files changed, 0 insertions(+), 0 deletions(-) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/FlinkPipelineExecutionEnvironment.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/FlinkPipelineOptions.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/FlinkPipelineRunner.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/FlinkRunnerResult.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/examples/TFIDF.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/examples/WordCount.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/examples/streaming/AutoComplete.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/examples/streaming/JoinExamples.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/examples/streaming/KafkaWindowedWordCountExample.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/examples/streaming/WindowedWordCount.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/io/ConsoleIO.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/FlinkBatchPipelineTranslator.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/FlinkBatchTransformTranslators.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/FlinkBatchTranslationContext.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/FlinkPipelineTranslator.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/FlinkStreamingPipelineTranslator.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/FlinkStreamingTransformTranslators.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/FlinkStreamingTranslationContext.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/functions/FlinkCoGroupKeyedListAggregator.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/functions/FlinkCreateFunction.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/functions/FlinkDoFnFunction.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/functions/FlinkKeyedListAggregationFunction.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/functions/FlinkMultiOutputDoFnFunction.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/functions/FlinkMultiOutputPruningFunction.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/functions/FlinkPartialReduceFunction.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/functions/FlinkReduceFunction.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/functions/UnionCoder.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/types/CoderComparator.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/types/CoderTypeInformation.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/types/CoderTypeSerializer.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/types/InspectableByteArrayOutputStream.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/types/KvCoderComperator.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/types/KvCoderTypeInformation.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/types/VoidCoderTypeSerializer.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/CombineFnAggregatorWrapper.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/DataInputViewWrapper.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/DataOutputViewWrapper.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/SerializableFnAggregatorWrapper.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/SinkOutputFormat.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/SourceInputFormat.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/SourceInputSplit.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/FlinkParDoBoundWrapper.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/io/UnboundedFlinkSource.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/io/UnboundedSocketSource.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/io/UnboundedSourceWrapper.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/state/FlinkStateInternals.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/state/StateCheckpointReader.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/state/StateCheckpointUtils.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/state/StateCheckpointWriter.java (100%) rename runners/flink/src/main/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/translation/wrappers/streaming/state/StateType.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/AvroITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/FlattenizeITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/FlinkTestPipeline.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/JoinExamplesITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/MaybeEmptyTestITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/ParDoMultiOutputITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/ReadSourceITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/RemoveDuplicatesEmptyITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/RemoveDuplicatesITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/SideInputITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/TfIdfITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/WordCountITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/WordCountJoin2ITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/WordCountJoin3ITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/WriteSinkITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/streaming/GroupAlsoByWindowTest.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/streaming/GroupByNullKeyTest.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/streaming/StateSerializationTest.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/streaming/TopWikipediaSessionsITCase.java (100%) rename runners/flink/src/test/java/{com/dataartisans/flink/dataflow => org/apache/beam/runners/flink}/util/JoinExamples.java (100%) diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java similarity index 100% rename from runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java rename to runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java similarity index 100% rename from runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java rename to runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java From 071347e3a3dc756951b0a744597159b980372b80 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 2 Mar 2016 23:21:57 +0100 Subject: [PATCH 146/149] [flink] update license headers --- runners/flink/LICENSE | 202 ------------------ .../FlinkPipelineExecutionEnvironment.java | 12 +- .../runners/flink/FlinkPipelineOptions.java | 12 +- .../runners/flink/FlinkPipelineRunner.java | 12 +- .../beam/runners/flink/FlinkRunnerResult.java | 12 +- .../runners/flink/examples/WordCount.java | 12 +- .../KafkaWindowedWordCountExample.java | 12 +- .../examples/streaming/WindowedWordCount.java | 12 +- .../beam/runners/flink/io/ConsoleIO.java | 12 +- .../FlinkBatchPipelineTranslator.java | 14 +- .../FlinkBatchTransformTranslators.java | 14 +- .../FlinkBatchTranslationContext.java | 14 +- .../translation/FlinkPipelineTranslator.java | 12 +- .../FlinkStreamingPipelineTranslator.java | 12 +- .../FlinkStreamingTransformTranslators.java | 12 +- .../FlinkStreamingTranslationContext.java | 12 +- .../FlinkCoGroupKeyedListAggregator.java | 12 +- .../functions/FlinkCreateFunction.java | 12 +- .../functions/FlinkDoFnFunction.java | 12 +- .../FlinkKeyedListAggregationFunction.java | 12 +- .../FlinkMultiOutputDoFnFunction.java | 12 +- .../FlinkMultiOutputPruningFunction.java | 12 +- .../functions/FlinkPartialReduceFunction.java | 14 +- .../functions/FlinkReduceFunction.java | 14 +- .../translation/types/CoderComparator.java | 14 +- .../types/CoderTypeInformation.java | 14 +- .../types/CoderTypeSerializer.java | 14 +- .../InspectableByteArrayOutputStream.java | 14 +- .../translation/types/KvCoderComperator.java | 14 +- .../types/KvCoderTypeInformation.java | 14 +- .../types/VoidCoderTypeSerializer.java | 14 +- .../wrappers/CombineFnAggregatorWrapper.java | 14 +- .../wrappers/DataInputViewWrapper.java | 14 +- .../wrappers/DataOutputViewWrapper.java | 14 +- .../SerializableFnAggregatorWrapper.java | 14 +- .../wrappers/SinkOutputFormat.java | 14 +- .../wrappers/SourceInputFormat.java | 14 +- .../wrappers/SourceInputSplit.java | 14 +- .../streaming/FlinkAbstractParDoWrapper.java | 12 +- .../FlinkGroupAlsoByWindowWrapper.java | 12 +- .../streaming/FlinkGroupByKeyWrapper.java | 12 +- .../FlinkParDoBoundMultiWrapper.java | 12 +- .../streaming/FlinkParDoBoundWrapper.java | 12 +- .../io/FlinkStreamingCreateFunction.java | 12 +- .../streaming/io/UnboundedFlinkSource.java | 12 +- .../streaming/io/UnboundedSocketSource.java | 12 +- .../streaming/io/UnboundedSourceWrapper.java | 12 +- .../state/AbstractFlinkTimerInternals.java | 12 +- .../streaming/state/FlinkStateInternals.java | 12 +- .../state/StateCheckpointReader.java | 12 +- .../streaming/state/StateCheckpointUtils.java | 12 +- .../state/StateCheckpointWriter.java | 12 +- .../wrappers/streaming/state/StateType.java | 12 +- .../flink/src/main/resources/log4j.properties | 24 ++- .../apache/beam/runners/flink/AvroITCase.java | 12 +- .../beam/runners/flink/FlattenizeITCase.java | 12 +- .../beam/runners/flink/FlinkTestPipeline.java | 12 +- .../runners/flink/JoinExamplesITCase.java | 12 +- .../runners/flink/MaybeEmptyTestITCase.java | 12 +- .../runners/flink/ParDoMultiOutputITCase.java | 12 +- .../beam/runners/flink/ReadSourceITCase.java | 12 +- .../flink/RemoveDuplicatesEmptyITCase.java | 12 +- .../runners/flink/RemoveDuplicatesITCase.java | 12 +- .../beam/runners/flink/SideInputITCase.java | 12 +- .../beam/runners/flink/TfIdfITCase.java | 12 +- .../beam/runners/flink/WordCountITCase.java | 12 +- .../runners/flink/WordCountJoin2ITCase.java | 12 +- .../runners/flink/WordCountJoin3ITCase.java | 12 +- .../beam/runners/flink/WriteSinkITCase.java | 12 +- .../streaming/GroupAlsoByWindowTest.java | 12 +- .../flink/streaming/GroupByNullKeyTest.java | 12 +- .../streaming/StateSerializationTest.java | 12 +- .../streaming/TopWikipediaSessionsITCase.java | 12 +- .../beam/runners/flink/util/JoinExamples.java | 12 +- 74 files changed, 517 insertions(+), 611 deletions(-) delete mode 100644 runners/flink/LICENSE diff --git a/runners/flink/LICENSE b/runners/flink/LICENSE deleted file mode 100644 index e09346158473..000000000000 --- a/runners/flink/LICENSE +++ /dev/null @@ -1,202 +0,0 @@ - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "{}" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright {yyyy} {name of copyright owner} - - Licensed 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. - diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java index 02a49b9b0856..8825ed36dee0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index bf833534263d..2f4b3ea47457 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java index 3c33d2088889..fe773d98ad39 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java index c2329a62482b..8fd08ec09132 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java index ba4630164dbf..7d12fedab294 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java index fa0c8e9b7bea..55cdc225b998 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java index 6af044d18d5f..7eb69ba870a2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java index cd25ba3d5735..71e3b54b3d57 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java index 520142357f28..9b47a08339b1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index f33e4f527476..48c783d39c9d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java index fd99833f056b..22943183db86 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java index efe217f9c1c8..9407bf564aef 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java index 21a8133dae98..60fba0f0815f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java index 10838485d9c2..bdefeaf80a7c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java index 83b19eee0415..f6bdecd88929 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java index 0e855addf9d3..d5562b8653ff 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java index 32efe4120226..56af39758ea8 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index 9e8b7cd2548d..fe77e6434f64 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java index 56fa88da649a..f92f888734d0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java index bd8a968f0d72..ca667ee4e338 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java index 3e1cb65ed4ee..37de37e4d68c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index 1ff06ba81b18..2de681b54739 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java index 580ac01b928b..29193a2a8e05 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java index ecfb95d0d535..12490363040e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java index 8880b4824339..f9d4dcd20cab 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java index 481ee3124e14..4e81054d1c9b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java index 619fa55b9251..36b5ba319180 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java index 4599c6ac4c0d..3912295afb0b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java index 7a0d9993209b..8862d48b9f36 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java index c7b6ea286077..8bc362002707 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java index 815765c4ac67..445d41129917 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java index b56a90ef31fd..6a3cf507be30 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java index 513d7f87986c..6bd2240d567c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java index 0d03f9f2307c..440958648309 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java index d0423b923cad..4c2475dbf4da 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java index 2d62416ab92a..cd5cd40768af 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java index 1b45ad7afdb0..cde2b3571c71 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java @@ -1,14 +1,14 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index e2ceae6282c9..10c8bbf07a04 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java index 906a39940c7d..e115a15cc90a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java index 61953a69e5f1..1a6a665858c2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java index cdf23f6124ab..df7f95355c19 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java index 3357cd5f7fdc..2ed56203b33a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java index 2599e88c51f7..f6c243fe1230 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java index ddbc99390174..2857efd40a23 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java index a24964a919cf..1389e9d98cdb 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index 7c1ccdf92a24..97084cfc7ab8 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java index 7accf09ff621..fc759486dc58 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java index 84007af09986..6cf46e5344b3 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java index d73ac8c05749..5aadccdefe3d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java index 055a12ae25c4..b2dc33cab6b2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java index 738ce5fb8962..18e118a8900e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java index 8b206006a636..58497730dd27 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/main/resources/log4j.properties b/runners/flink/src/main/resources/log4j.properties index c88022c19960..4daaad1e22a5 100644 --- a/runners/flink/src/main/resources/log4j.properties +++ b/runners/flink/src/main/resources/log4j.properties @@ -1,17 +1,19 @@ ################################################################################ -# Copyright 2015 Stephan Ewen, dataArtisans +# 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 # -# Licensed 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 # -# 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. +# 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. ################################################################################ log4j.rootLogger=INFO,console diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java index eaa597988692..5b32d54513be 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java index 79eb16329383..5ae0e832fd8b 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java index 2dcebdef5520..aadda24b4c84 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java index 11b6ce40171c..f60056ded067 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java index e39b81def51f..199602c6c3e0 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java index 08e532330f34..403de29600b7 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java index 72024175e1d4..323c41ba0da6 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java index dc82d7d21df2..524554aa8d5c 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java index 78b48b5ecc7d..54e92aa9ec39 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java index 5cd7d78e577e..7f73b8309605 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java index ceb0a3f1c053..8722feefb960 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java index c2b6fdd55ead..8ca978e79792 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java index d78434b2427c..e73c4568df0f 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java index 083627943768..6b57d771070b 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java index 497a5bb8868e..dfa15ce62b19 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java index 27ddc836ee90..880da59792af 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java index 80d78b94471f..63e0bcf718f2 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java index e6c5ae2fd17d..77a8de65082e 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java index f0b93a00d205..83c1661fcca6 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java index 620dace09d1c..e850dd6a451e 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java @@ -1,9 +1,11 @@ /* - * Copyright 2015 Data Artisans GmbH - * - * Licensed 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 + * 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 * From 4de42e48084d7d37ccb8b6d363041bc93c4364a5 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 2 Mar 2016 23:51:38 +0100 Subject: [PATCH 147/149] [flink] update README --- runners/flink/README.md | 60 ++++++++++++++++++++--------------------- 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/runners/flink/README.md b/runners/flink/README.md index 499ed6d0b72a..0fee6f09cbf5 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -1,17 +1,17 @@ -Flink-Dataflow --------------- +Flink Beam Runner (Flink-Runner) +------------------------------- -Flink-Dataflow is a Runner for Google Dataflow (aka Apache Beam) which enables you to -run Dataflow programs with Flink. It integrates seamlessly with the Dataflow -API, allowing you to execute Dataflow programs in streaming or batch mode. +Flink-Runner is a Runner for Apache Beam which enables you to +run Beam dataflows with Flink. It integrates seamlessly with the Beam +API, allowing you to execute Apache Beam programs in streaming or batch mode. ## Streaming -### Full Dataflow Windowing and Triggering Semantics +### Full Beam Windowing and Triggering Semantics -The Flink Dataflow Runner supports *Event Time* allowing you to analyze data with respect to its +The Flink Beam Runner supports *Event Time* allowing you to analyze data with respect to its associated timestamp. It handles out-or-order and late-arriving elements. You may leverage the full -power of the Dataflow windowing semantics like *time-based*, *sliding*, *tumbling*, or *count* +power of the Beam windowing semantics like *time-based*, *sliding*, *tumbling*, or *count* windows. You may build *session* windows which allow you to keep track of events associated with each other. @@ -27,7 +27,7 @@ and sinks or use the provided support for Apache Kafka. ### Seamless integration -To execute a Dataflow program in streaming mode, just enable streaming in the `PipelineOptions`: +To execute a Beam program in streaming mode, just enable streaming in the `PipelineOptions`: options.setStreaming(true); @@ -52,7 +52,7 @@ and sinks. ## Features -The Flink Dataflow Runner maintains as much compatibility with the Dataflow API as possible. We +The Flink Beam Runner maintains as much compatibility with the Beam API as possible. We support transformations on data like: - Grouping @@ -66,25 +66,25 @@ support transformations on data like: # Getting Started -To get started using Flink-Dataflow, we first need to install the latest version. +To get started using the Flink Runner, we first need to install the latest version. -## Install Flink-Dataflow ## +## Install Flink-Runner ## -To retrieve the latest version of Flink-Dataflow, run the following command +To retrieve the latest version of Flink-Runner, run the following command - git clone https://github.com/dataArtisans/flink-dataflow + git clone https://github.com/apache/incubator-beam -Then switch to the newly created directory and run Maven to build the Dataflow runner: +Then switch to the newly created directory and run Maven to build the Beam runner: - cd flink-dataflow + cd incubator-beam mvn clean install -DskipTests -Flink-Dataflow is now installed in your local maven repository. +Flink-Runner is now installed in your local maven repository. ## Executing an example Next, let's run the classic WordCount example. It's semantically identically to -the example provided with Google Dataflow. Only this time, we chose the +the example provided with ApacheBeam. Only this time, we chose the `FlinkPipelineRunner` to execute the WordCount on top of Flink. Here's an excerpt from the WordCount class file: @@ -113,15 +113,15 @@ Then let's run the included WordCount locally on your machine: mvn exec:exec -Dinput=kinglear.txt -Doutput=wordcounts.txt -Congratulations, you have run your first Google Dataflow program on top of Apache Flink! +Congratulations, you have run your first ApacheBeam program on top of Apache Flink! -# Running Dataflow programs on a Flink cluster +# Running Beam programs on a Flink cluster -You can run your Dataflow program on an Apache Flink cluster. Please start off by creating a new +You can run your Beam program on an Apache Flink cluster. Please start off by creating a new Maven project. - mvn archetype:generate -DgroupId=com.mycompany.dataflow -DartifactId=dataflow-test \ + mvn archetype:generate -DgroupId=com.mycompany.beam -DartifactId=beam-test \ -DarchetypeArtifactId=maven-archetype-quickstart -DinteractiveMode=false The contents of the root `pom.xml` should be slightly changed aftewards (explanation below): @@ -133,14 +133,14 @@ The contents of the root `pom.xml` should be slightly changed aftewards (explana xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - com.mycompany.dataflow - dataflow-test + com.mycompany.beam + beam-test 1.0 - com.dataartisans - flink-dataflow + org.apache.beam + flink-runner 0.2 @@ -182,13 +182,13 @@ The contents of the root `pom.xml` should be slightly changed aftewards (explana The following changes have been made: -1. The Flink Dataflow Runner was added as a dependency. +1. The Flink Beam Runner was added as a dependency. 2. The Maven Shade plugin was added to build a fat jar. -A fat jar is necessary if you want to submit your Dataflow code to a Flink cluster. The fat jar -includes your program code but also Dataflow code which is necessary during runtime. Note that this -step is necessary because the Dataflow Runner is not part of Flink. +A fat jar is necessary if you want to submit your Beam code to a Flink cluster. The fat jar +includes your program code but also Beam code which is necessary during runtime. Note that this +step is necessary because the Beam Runner is not part of Flink. You can then build the jar using `mvn clean package`. Please submit the fat jar in the `target` folder to the Flink cluster using the command-line utility like so: From 6747817f2a03b912431b7e6804aa05f40b72594c Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 4 Mar 2016 14:51:14 +0100 Subject: [PATCH 148/149] [flink] [cleanup] delete .gitignore --- runners/flink/.gitignore | 19 ------------------- 1 file changed, 19 deletions(-) delete mode 100644 runners/flink/.gitignore diff --git a/runners/flink/.gitignore b/runners/flink/.gitignore deleted file mode 100644 index e40b671f8af5..000000000000 --- a/runners/flink/.gitignore +++ /dev/null @@ -1,19 +0,0 @@ -dependency-reduced-pom.xml -.cache -.classpath -.idea -.metadata -.settings -.project -.version.properties -filter.properties -target -tmp -*.class -*.iml -*.swp -*.jar -*.log -.DS_Store -_site -\? \ No newline at end of file From 460b58f5379f7de01e0756da29c81ddb414edcf9 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 4 Mar 2016 14:52:35 +0100 Subject: [PATCH 149/149] [travis] go through all Maven phases (except deploy) - mvn verify ensures integration tests are also run --- .travis.yml | 2 +- runners/flink/.travis.yml | 14 -------------- 2 files changed, 1 insertion(+), 15 deletions(-) delete mode 100644 runners/flink/.travis.yml diff --git a/.travis.yml b/.travis.yml index 52e1d3a5cbd2..973618b10696 100644 --- a/.travis.yml +++ b/.travis.yml @@ -31,5 +31,5 @@ install: script: - travis_retry mvn versions:set -DnewVersion=manual_build - - travis_retry mvn $MAVEN_OVERRIDE install -U + - travis_retry mvn $MAVEN_OVERRIDE verify -U - travis_retry travis/test_wordcount.sh diff --git a/runners/flink/.travis.yml b/runners/flink/.travis.yml deleted file mode 100644 index 4780b56c0bd3..000000000000 --- a/runners/flink/.travis.yml +++ /dev/null @@ -1,14 +0,0 @@ -sudo: false - -language: java - -jdk: - - oraclejdk7 - - oraclejdk8 - - openjdk7 - # not supported yet :( - #- openjdk8 - -install: (git clone --depth 1 https://github.com/GoogleCloudPlatform/DataflowJavaSDK.git sdk && cd sdk && mvn clean install -DskipTests) - -script: mvn verify \ No newline at end of file