diff --git a/examples/java/README.md b/examples/java/README.md index cbcd01fc0f1c..4533303a23be 100644 --- a/examples/java/README.md +++ b/examples/java/README.md @@ -44,7 +44,7 @@ the same pipeline on fully managed resources in Google Cloud Platform: mvn compile exec:java -pl examples \ -Dexec.mainClass=com.google.cloud.dataflow.examples.WordCount \ -Dexec.args="--project= \ - --stagingLocation= \ + --tempLocation= \ --runner=BlockingDataflowPipelineRunner" Make sure to use your project id, not the project number or the descriptive name. @@ -66,7 +66,7 @@ Platform: java -cp examples/target/google-cloud-dataflow-java-examples-all-bundled-.jar \ com.google.cloud.dataflow.examples.WordCount \ --project= \ - --stagingLocation= \ + --tempLocation= \ --runner=BlockingDataflowPipelineRunner Other examples can be run similarly by replacing the `WordCount` class path with the example classpath, e.g. diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 342986f3f906..3457f71aa96f 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -60,23 +60,6 @@ org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - @@ -150,7 +133,6 @@ org.apache.maven.plugins maven-shade-plugin - 2.4.1 package @@ -182,20 +164,6 @@ org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - @@ -232,7 +200,6 @@ org.apache.beam java-sdk-all - ${project.version} @@ -244,121 +211,61 @@ com.google.api-client google-api-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-dataflow - ${dataflow.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-bigquery - ${bigquery.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - org.apache.avro avro - ${avro.version} com.google.apis google-api-services-datastore-protobuf - ${datastore.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-pubsub - ${pubsub.version} - - - - com.google.guava - guava-jdk5 - - com.google.guava guava - ${guava.version} com.google.code.findbugs jsr305 - ${jsr305.version} joda-time joda-time - ${joda.version} org.slf4j slf4j-api - ${slf4j.version} org.slf4j slf4j-jdk14 - ${slf4j.version} runtime @@ -374,19 +281,16 @@ org.hamcrest hamcrest-all - ${hamcrest.version} junit junit - ${junit.version} org.mockito mockito-all - 1.10.19 test diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index 2e484651a634..11c56d132b0d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -68,7 +68,7 @@ * below, specify pipeline configuration: *
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
  * }
diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
index ec517c4bd69c..65d450dfce23 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
@@ -50,26 +50,26 @@
  *   4. Writing data to Cloud Storage as text files
  * 
* - *

To execute this pipeline, first edit the code to set your project ID, the staging + *

To execute this pipeline, first edit the code to set your project ID, the temp * location, and the output location. The specified GCS bucket(s) must already exist. * - *

Then, run the pipeline as described in the README. It will be deployed and run using the - * Dataflow service. No args are required to run the pipeline. You can see the results in your + *

Then, run the pipeline as described in the README. It will be deployed and run with the + * selected runner. No args are required to run the pipeline. You can see the results in your * output bucket in the GCS browser. */ public class MinimalWordCount { public static void main(String[] args) { - // Create a DataflowPipelineOptions object. This object lets us set various execution + // Create a PipelineOptions object. This object lets us set various execution // options for our pipeline, such as the associated Cloud Platform project and the location // in Google Cloud Storage to stage files. DataflowPipelineOptions options = PipelineOptionsFactory.create() - .as(DataflowPipelineOptions.class); + .as(DataflowPipelineOptions.class); options.setRunner(BlockingDataflowPipelineRunner.class); // CHANGE 1/3: Your project ID is required in order to run your pipeline on the Google Cloud. options.setProject("SET_YOUR_PROJECT_ID_HERE"); // CHANGE 2/3: Your Google Cloud Storage path is required for staging local files. - options.setStagingLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_STAGING_DIRECTORY"); + options.setTempLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_TEMP_DIRECTORY"); // Create the Pipeline object with the options we defined above. Pipeline p = Pipeline.create(options); diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java index ecb837e46eac..4d019bbf81c2 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java @@ -83,7 +83,7 @@ *

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

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java index 364d075c2b21..f5611d34146a 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -17,7 +17,6 @@ */ package org.apache.beam.examples; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; @@ -37,9 +36,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - /** - * An example that counts words in Shakespeare and includes Dataflow best practices. + * An example that counts words in Shakespeare and includes Beam best practices. * *

This class, {@link WordCount}, is the second in a series of four successively more detailed * 'word count' examples. You may first want to take a look at {@link MinimalWordCount}. @@ -56,13 +54,13 @@ * *

New Concepts: *

- *   1. Executing a Pipeline both locally and using the Dataflow service
+ *   1. Executing a Pipeline both locally and using the selected runner
  *   2. Using ParDo with static DoFns defined out-of-line
  *   3. Building a composite transform
  *   4. Defining your own pipeline options
  * 
* - *

Concept #1: you can execute this pipeline either locally or using the Dataflow service. + *

Concept #1: you can execute this pipeline either locally or using the selected runner. * These are now command-line options and not hard-coded as they were in the MinimalWordCount * example. * To execute this pipeline locally, specify general pipeline configuration: @@ -78,7 +76,7 @@ *

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

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
@@ -173,17 +171,16 @@ public static interface WordCountOptions extends PipelineOptions { void setOutput(String value); /** - * Returns "gs://${YOUR_STAGING_DIRECTORY}/counts.txt" as the default destination. + * Returns "gs://${YOUR_TEMP_DIRECTORY}/counts.txt" as the default destination. */ public static class OutputFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { - DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); - if (dataflowOptions.getStagingLocation() != null) { - return GcsPath.fromUri(dataflowOptions.getStagingLocation()) + if (options.getTempLocation() != null) { + return GcsPath.fromUri(options.getTempLocation()) .resolve("counts.txt").toString(); } else { - throw new IllegalArgumentException("Must specify --output or --stagingLocation"); + throw new IllegalArgumentException("Must specify --output or --tempLocation"); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java index 7ac71d300b99..fb4f3bfb413c 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java @@ -51,7 +51,6 @@ import com.google.api.services.pubsub.model.Subscription; import com.google.api.services.pubsub.model.Topic; import com.google.common.base.Strings; -import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.util.concurrent.Uninterruptibles; @@ -116,7 +115,7 @@ public void setup() throws IOException { Thread.currentThread().interrupt(); // Ignore InterruptedException } - Throwables.propagate(lastException); + throw new RuntimeException(lastException); } /** diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java index 2f2283cdb41f..c83a898aecdc 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java @@ -81,7 +81,7 @@ * specify pipeline configuration: *
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=DataflowPipelineRunner
  *   --inputFile=gs://path/to/input*.txt
  * }
@@ -90,7 +90,7 @@ * specify pipeline configuration: *
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=DataflowPipelineRunner
  *   --inputFile=gs://YOUR_INPUT_DIRECTORY/*.txt
  *   --streaming
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
index 4b0db93d3e6e..1f1885fa0b6f 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
@@ -79,7 +79,7 @@
  * 

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

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * and an output prefix on GCS:
  *   --output=gs://YOUR_OUTPUT_PREFIX
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
index e7090fdf2f57..c483608ceb0e 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
@@ -59,7 +59,7 @@
  * 

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

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java index 47631aaf7274..80a2f25569ab 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/BigQueryTornadoes.java @@ -60,7 +60,7 @@ *

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

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java index 1577b77bc279..505989c6fb84 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/CombinePerKeyExamples.java @@ -67,7 +67,7 @@ *

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

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://
+ *   --tempLocation=gs://
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index 021fc14616b1..bfaecdf897ae 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -69,7 +69,7 @@ * }
* *

To run this example using Dataflow service, you must additionally - * provide either {@literal --stagingLocation} or {@literal --tempLocation}, and + * provide either {@literal --tempLocation} or {@literal --tempLocation}, and * select one of the Dataflow pipeline runners, eg * {@literal --runner=BlockingDataflowPipelineRunner}. * diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java index fe2bbc8573c3..37ddb4b8c29c 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java @@ -17,7 +17,6 @@ */ package org.apache.beam.examples.cookbook; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; @@ -43,7 +42,7 @@ * *

To execute this pipeline using the Dataflow service, specify pipeline configuration: * --project=YOUR_PROJECT_ID - * --stagingLocation=gs://YOUR_STAGING_DIRECTORY + * --tempLocation=gs://YOUR_TEMP_DIRECTORY * --runner=BlockingDataflowPipelineRunner * and an output prefix on GCS: * --output=gs://YOUR_OUTPUT_PREFIX @@ -69,16 +68,15 @@ private static interface Options extends PipelineOptions { String getOutput(); void setOutput(String value); - /** Returns gs://${STAGING_LOCATION}/"deduped.txt". */ + /** Returns gs://${TEMP_LOCATION}/"deduped.txt". */ public static class OutputFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { - DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); - if (dataflowOptions.getStagingLocation() != null) { - return GcsPath.fromUri(dataflowOptions.getStagingLocation()) + if (options.getTempLocation() != null) { + return GcsPath.fromUri(options.getTempLocation()) .resolve("deduped.txt").toString(); } else { - throw new IllegalArgumentException("Must specify --output or --stagingLocation"); + throw new IllegalArgumentException("Must specify --output or --tempLocation"); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java index 597ef701d434..1c6a6e1e9266 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java @@ -72,7 +72,7 @@ *

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

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java index 7cf138f4352a..8e642f87fc7b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java @@ -55,7 +55,7 @@ *

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

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java index 3bc120855b44..24809672e6dd 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/MaxPerKeyExamples.java @@ -61,7 +61,7 @@ *

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

{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  * }
  * 
diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index 56ca98c1a6b7..503445e6fbbd 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -18,13 +18,7 @@ package org.apache.beam.examples; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - import org.apache.beam.examples.WordCount.WordCountOptions; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.runners.dataflow.testing.TestDataflowPipelineRunner; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; @@ -34,6 +28,8 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import java.util.Date; + /** * End-to-end tests of WordCount. */ @@ -43,8 +39,7 @@ public class WordCountIT { /** * Options for the WordCount Integration Test. */ - public static interface WordCountITOptions extends TestPipelineOptions, - WordCountOptions, DataflowPipelineOptions { + public interface WordCountITOptions extends TestPipelineOptions, WordCountOptions { } @Test @@ -52,13 +47,8 @@ public void testE2EWordCount() throws Exception { PipelineOptionsFactory.register(WordCountITOptions.class); WordCountITOptions options = TestPipeline.testingPipelineOptions().as(WordCountITOptions.class); options.setOutput(Joiner.on("/").join(new String[]{options.getTempRoot(), - options.getJobName(), "output", "results"})); + String.format("WordCountIT-%tF-% org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - @@ -121,7 +90,6 @@ org.apache.beam java-sdk-all - ${project.version} @@ -139,114 +107,66 @@ com.google.guava guava - ${guava.version} org.slf4j slf4j-api - ${slf4j.version} org.apache.avro avro - ${avro.version} joda-time joda-time - ${joda.version} org.hamcrest hamcrest-all - ${hamcrest.version} test org.mockito mockito-all - 1.10.19 test junit junit - ${junit.version} test com.google.apis google-api-services-bigquery - ${bigquery.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.oauth-client google-oauth-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-pubsub - ${pubsub.version} - - - - com.google.guava - guava-jdk5 - - com.google.api-client google-api-client - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - + + diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java index 12dfdf9abebb..6187a1bb1e19 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java @@ -83,7 +83,7 @@ * like this: *
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  *   --dataset=YOUR-DATASET
  *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
index b516a322455b..845c56fa0336 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
@@ -59,7 +59,7 @@
  * like this:
  * 
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  *   --dataset=YOUR-DATASET
  * }
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
index a5d9fb93fb03..0b10f07f93ff 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
@@ -80,7 +80,7 @@
  * like this:
  * 
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  *   --dataset=YOUR-DATASET
  *   --topic=projects/YOUR-PROJECT/topics/YOUR-TOPIC
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
index 6a6c1cf03866..866adefdb886 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
@@ -62,7 +62,7 @@
  * the pipeline configuration like this:
  * 
{@code
  *   --project=YOUR_PROJECT_ID
- *   --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
  *   --runner=BlockingDataflowPipelineRunner
  *   --dataset=YOUR-DATASET
  * }
diff --git a/pom.xml b/pom.xml
index 91c6da5ea4e1..fda7a24bcd00 100644
--- a/pom.xml
+++ b/pom.xml
@@ -101,19 +101,25 @@
 
     
     1.7.7
-    v2-rev248-1.21.0
+    v2-rev292-1.21.0
     0.0.2
     v2-rev6-1.21.0
     v1b3-rev22-1.21.0
     0.5.160222
     v1beta2-rev1-4.0.0
+    1.0-rc2
+    1.1
     1.21.0
+    1.4.3
+    0.5.160304
     19.0
+    0.12.0
     1.3
-    2.7.0
+    2.7.2
     2.4
     3.0.1
     4.11
+    1.9.5
     3.0.0-beta-1
     v1-rev7-1.21.0
     1.7.14
@@ -126,19 +132,79 @@
 
   
     sdks
-    
+    
     runners/google-cloud-dataflow-java
     runners
-    
+    
     sdks/java/maven-archetypes
     examples
   
 
   
+
+    
+    
+      release
+      
+        
+          
+            
+              org.apache.maven.plugins
+              maven-javadoc-plugin
+              
+                
+                  javadoc
+                  package
+                  
+                    jar
+                  
+                
+              
+            
+
+            
+              org.apache.maven.plugins
+              maven-source-plugin
+              
+                
+                  attach-sources
+                  package
+                  
+                    jar-no-fork
+                  
+                
+                
+                  attach-test-sources
+                  package
+                  
+                    test-jar-no-fork
+                  
+                
+              
+            
+
+            
+              org.apache.maven.plugins
+              maven-jar-plugin
+              
+                
+                  default-test-jar
+                  
+                    test-jar
+                  
+                
+              
+            
+          
+        
+      
+    
+
     
       doclint-java8-disable
       
@@ -148,6 +214,7 @@
         -Xdoclint:-missing
       
     
+
     
       src
       
@@ -179,6 +246,371 @@
     
   
 
+  
+    
+
+      
+        org.apache.beam
+        java-sdk-all
+        ${project.version}
+      
+
+      
+        org.apache.beam
+        runners-core
+        ${project.version}
+      
+
+      
+        org.apache.beam
+        google-cloud-dataflow-java-runner
+        ${project.version}
+      
+
+      
+        org.apache.beam
+        java-examples-all
+        ${project.version}
+      
+
+      
+        io.grpc
+        grpc-all
+        ${grpc.version}
+      
+
+      
+        com.google.api-client
+        google-api-client
+        ${google-clients.version}
+        
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.oauth-client
+        google-oauth-client
+        ${google-clients.version}
+        
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.oauth-client
+        google-oauth-client-java6
+        ${google-clients.version}
+        
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.http-client
+        google-http-client
+        ${google-clients.version}
+        
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.http-client
+        google-http-client-jackson
+        ${google-clients.version}
+        
+          
+            com.google.guava
+            guava-jdk5
+          
+          
+            org.codehaus.jackson
+            jackson-core-asl
+          
+        
+      
+
+      
+        com.google.http-client
+        google-http-client-jackson2
+        ${google-clients.version}
+        
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.http-client
+        google-http-client-protobuf
+        ${google-clients.version}
+        
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.apis
+        google-api-services-bigquery
+        ${bigquery.version}
+        
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.apis
+        google-api-services-pubsub
+        ${pubsub.version}
+        
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.apis
+        google-api-services-storage
+        ${storage.version}
+        
+          
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.apis
+        google-api-services-datastore-protobuf
+        ${datastore.version}
+        
+          
+          
+            com.google.guava
+            guava-jdk5
+          
+          
+          
+            com.google.http-client
+            google-http-client
+          
+          
+            com.google.api-client
+            google-api-client
+          
+          
+            com.google.oauth-client
+            google-oauth-client
+          
+          
+            com.google.http-client
+            google-http-client-jackson
+          
+          
+            com.google.http-client
+            google-http-client-protobuf
+          
+        
+      
+
+      
+        com.google.auto.service
+        auto-service
+        ${google-auto-service.version}
+      
+
+      
+        com.google.auto.value
+        auto-value
+        ${google-auto-value.version}
+      
+
+      
+        com.google.code.findbugs
+        jsr305
+        ${jsr305.version}
+      
+
+      
+        com.google.cloud.bigdataoss
+        util
+        ${google-cloud-bigdataoss.version}
+      
+      
+        com.google.cloud.bigdataoss
+        gcsio
+        ${google-cloud-bigdataoss.version}
+      
+
+      
+        com.google.apis
+        google-api-services-dataflow
+        ${dataflow.version}
+        
+          
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.apis
+        google-api-services-clouddebugger
+        ${clouddebugger.version}
+        
+          
+          
+            com.google.guava
+            guava-jdk5
+          
+        
+      
+
+      
+        com.google.protobuf
+        protobuf-java
+        ${protobuf.version}
+      
+
+      
+        com.fasterxml.jackson.core
+        jackson-core
+        ${jackson.version}
+      
+      
+        com.fasterxml.jackson.core
+        jackson-annotations
+        ${jackson.version}
+      
+      
+        com.fasterxml.jackson.core
+        jackson-databind
+        ${jackson.version}
+      
+      
+        com.fasterxml.jackson.module
+        jackson-module-scala_2.10
+        ${jackson.version}
+      
+
+      
+        org.apache.avro
+        avro
+        ${avro.version}
+      
+
+      
+        com.google.guava
+        guava
+        ${guava.version}
+      
+
+      
+        joda-time
+        joda-time
+        ${joda.version}
+      
+
+      
+        org.slf4j
+        slf4j-api
+        ${slf4j.version}
+      
+
+      
+
+      
+        org.apache.beam
+        java-sdk-all
+        ${project.version}
+        test-jar
+        test
+      
+
+      
+        org.apache.beam
+        java-sdk-all
+        ${project.version}
+        tests
+        test
+      
+
+      
+        org.hamcrest
+        hamcrest-all
+        ${hamcrest.version}
+        test
+      
+
+      
+        junit
+        junit
+        ${junit.version}
+        test
+      
+
+      
+        org.slf4j
+        slf4j-jdk14
+        ${slf4j.version}
+        test
+      
+
+      
+        com.google.guava
+        guava-testlib
+        ${guava.version}
+        test
+      
+
+      
+        org.mockito
+        mockito-all
+        ${mockito.version}
+        test
+      
+
+      
+        com.google.cloud.dataflow
+        google-cloud-dataflow-java-proto-library-all
+        ${google-cloud-dataflow-java-proto-library-all.version}
+        test
+      
+
+    
+  
+
   
     
       
@@ -235,7 +667,7 @@
               -Xlint:-try
               -Xlint:-unchecked
               -Xlint:-varargs
-              
+              
               
               
             
@@ -249,6 +681,20 @@
           org.apache.maven.plugins
           maven-jar-plugin
           2.5
+          
+            
+              default-jar
+              
+                jar
+              
+            
+          
+        
+
+        
+          org.apache.maven.plugins
+          maven-source-plugin
+          2.4
         
 
         
@@ -308,11 +754,11 @@
               
             
           
-        
-          
-            **/AutoValue_*.class
-          
-        
+          
+            
+              **/AutoValue_*.class
+            
+          
         
 
         
@@ -346,8 +792,8 @@
           2.19.1
         
 
-        
+        
         
           org.eclipse.m2e
           lifecycle-mapping
@@ -381,7 +827,7 @@
                     
                   
                   
-                    
+                    
                   
                 
                 
@@ -395,7 +841,7 @@
                     
                   
                   
-                    
+                    
                   
                 
               
@@ -419,6 +865,18 @@
           1.10
         
 
+        
+          org.apache.maven.plugins
+          maven-shade-plugin
+          2.4.1
+        
+
+        
+          org.apache.avro
+          avro-maven-plugin
+          ${avro.version}
+        
+
         
           org.apache.maven.plugins
           maven-release-plugin
@@ -432,6 +890,32 @@
         
       
     
+
+    
+      
+        org.apache.maven.plugins
+        maven-enforcer-plugin
+        1.3.1
+        
+          
+            enforce
+            
+              enforce
+            
+            
+              
+                
+                  [1.7,)
+                
+                
+                  [3.0.3,)
+                
+              
+            
+          
+        
+      
+    
   
 
   
diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml
new file mode 100644
index 000000000000..b6f6f2986925
--- /dev/null
+++ b/runners/core-java/pom.xml
@@ -0,0 +1,208 @@
+
+
+
+
+  4.0.0
+
+  
+    org.apache.beam
+    runners-parent
+    0.1.0-incubating-SNAPSHOT
+    ../pom.xml
+  
+
+  runners-core
+  Apache Beam :: Runners :: Core
+  Beam Runners Core provides utilities to aid runner authors.
+
+  jar
+
+  
+    
+      
+        org.apache.maven.plugins
+        maven-compiler-plugin
+      
+
+      
+        org.apache.maven.plugins
+        maven-dependency-plugin
+        
+          
+            analyze-only
+            
+              true
+            
+          
+        
+      
+
+      
+        org.apache.maven.plugins
+        maven-checkstyle-plugin
+      
+
+      
+        org.apache.maven.plugins
+        maven-jar-plugin
+      
+
+      
+      
+        org.apache.maven.plugins
+        maven-source-plugin
+      
+
+      
+        org.apache.maven.plugins
+        maven-javadoc-plugin
+        
+          Beam Runners Core utilities ${project.version} API
+          Beam Runners Core utilities for Java, version ${project.version}
+          ../javadoc/overview.html
+
+          org.apache.beam.runners.core
+          false
+          true
+          ]]>
+        
+        
+          
+            
+              jar
+            
+            package
+          
+        
+      
+
+      
+        org.apache.maven.plugins
+        maven-shade-plugin
+        2.4.1
+        
+          
+          
+            bundle-and-repackage
+            package
+            
+              shade
+            
+            
+              true
+              
+                
+                  com.google.guava:guava
+                
+              
+              
+                
+                  *:*
+                  
+                    META-INF/*.SF
+                    META-INF/*.DSA
+                    META-INF/*.RSA
+                  
+                
+              
+              
+                
+                
+                  com.google.common
+                  org.apache.beam.sdk.repackaged.com.google.common
+                
+                
+                  com.google.thirdparty
+                  org.apache.beam.sdk.repackaged.com.google.thirdparty
+                
+              
+            
+          
+
+          
+          
+            bundle-rest-without-repackaging
+            package
+            
+              shade
+            
+            
+              true
+              ${project.artifactId}-bundled-${project.version}
+              
+                
+                  com.google.guava:guava
+                
+              
+              
+                
+                  *:*
+                  
+                    META-INF/*.SF
+                    META-INF/*.DSA
+                    META-INF/*.RSA
+                  
+                
+              
+            
+          
+        
+      
+
+      
+      
+        org.jacoco
+        jacoco-maven-plugin
+      
+
+    
+  
+
+  
+    
+      org.apache.beam
+      java-sdk-all
+    
+
+    
+
+    
+    
+      org.hamcrest
+      hamcrest-all
+      test
+    
+
+    
+      junit
+      junit
+      test
+    
+
+    
+      org.slf4j
+      slf4j-jdk14
+      test
+    
+  
+
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java
similarity index 91%
rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java
rename to runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java
index c24f6daac371..73244f7eca21 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowViaWindowSetDoFn.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java
@@ -15,14 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.beam.sdk.util;
+package org.apache.beam.runners.core;
 
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor;
+import org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn;
+import org.apache.beam.sdk.util.KeyedWorkItem;
+import org.apache.beam.sdk.util.ReduceFnRunner;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.SystemReduceFn;
+import org.apache.beam.sdk.util.TimerInternals;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
+import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.KV;
 
diff --git a/runners/core-java/src/test/java/.placeholder b/runners/core-java/src/test/java/.placeholder
new file mode 100644
index 000000000000..e69de29bb2d1
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
index 12ba329db70c..bec9b70a8ffb 100644
--- a/runners/direct-java/pom.xml
+++ b/runners/direct-java/pom.xml
@@ -69,43 +69,12 @@
       
         org.apache.maven.plugins
         maven-jar-plugin
-        
-          
-            default-jar
-            
-              jar
-            
-          
-          
-            default-test-jar
-            
-              test-jar
-            
-          
-        
       
 
       
       
         org.apache.maven.plugins
         maven-source-plugin
-        2.4
-        
-          
-            attach-sources
-            verify
-            
-              jar-no-fork
-            
-          
-          
-            attach-test-sources
-            verify
-            
-              test-jar-no-fork
-            
-          
-        
       
 
       
@@ -181,20 +150,11 @@
             
           
         
-        
-          
-            
-              jar
-            
-            package
-          
-        
       
 
       
         org.apache.maven.plugins
         maven-shade-plugin
-        2.4.1
         
           
           
@@ -275,88 +235,60 @@
     
       org.apache.beam
       java-sdk-all
-      ${project.version}
     
 
     
-      com.google.http-client
-      google-http-client
-      ${google-clients.version}
-      
-        
-        
-          com.google.guava
-          guava-jdk5
-        
-      
+      org.apache.beam
+      runners-core
     
 
     
       com.google.http-client
       google-http-client-protobuf
-      ${google-clients.version}
-      
-        
-        
-          com.google.guava
-          guava-jdk5
-        
-      
       runtime
     
 
     
       com.google.guava
       guava
-      
-      ${guava.version}
     
 
     
       com.google.guava
       guava-testlib
-      ${guava.version}
       test
     
 
     
       joda-time
       joda-time
-      ${joda.version}
     
 
     
       com.google.code.findbugs
       jsr305
-      ${jsr305.version}
     
 
     
       com.fasterxml.jackson.core
       jackson-annotations
-      ${jackson.version}
     
 
     
       org.slf4j
       slf4j-api
-      ${slf4j.version}
     
 
     
     
       com.google.auto.service
       auto-service
-      1.0-rc2
       true
     
 
     
       com.google.auto.value
       auto-value
-      1.1
       provided
     
 
@@ -364,35 +296,30 @@
     
       org.hamcrest
       hamcrest-all
-      ${hamcrest.version}
       provided
     
 
     
       junit
       junit
-      ${junit.version}
       provided
     
 
     
       org.slf4j
       slf4j-jdk14
-      ${slf4j.version}
       test
     
 
     
       org.mockito
       mockito-all
-      1.10.19
       test
     
 
     
       org.apache.beam
       java-sdk-all
-      ${project.version}
       test-jar
       test
     
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
index 18af36337692..9f26e5a51677 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
@@ -210,16 +210,20 @@ public void awaitCompletion() throws Throwable {
   }
 
   /**
-   * The default {@link CompletionCallback}. The default completion callback is used to complete
-   * transform evaluations that are triggered due to the arrival of elements from an upstream
-   * transform, or for a source transform.
+   * The base implementation of {@link CompletionCallback} that provides implementations for
+   * {@link #handleResult(CommittedBundle, InProcessTransformResult)} and
+   * {@link #handleThrowable(CommittedBundle, Throwable)}, given an implementation of
+   * {@link #getCommittedResult(CommittedBundle, InProcessTransformResult)}.
    */
-  private class DefaultCompletionCallback implements CompletionCallback {
+  private abstract class CompletionCallbackBase implements CompletionCallback {
+    protected abstract CommittedResult getCommittedResult(
+        CommittedBundle inputBundle,
+        InProcessTransformResult result);
+
     @Override
-    public CommittedResult handleResult(
+    public final CommittedResult handleResult(
         CommittedBundle inputBundle, InProcessTransformResult result) {
-      CommittedResult committedResult =
-          evaluationContext.handleResult(inputBundle, Collections.emptyList(), result);
+      CommittedResult committedResult = getCommittedResult(inputBundle, result);
       for (CommittedBundle outputBundle : committedResult.getOutputs()) {
         allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
       }
@@ -227,18 +231,33 @@ public CommittedResult handleResult(
     }
 
     @Override
-    public void handleThrowable(CommittedBundle inputBundle, Throwable t) {
+    public final void handleThrowable(CommittedBundle inputBundle, Throwable t) {
       allUpdates.offer(ExecutorUpdate.fromThrowable(t));
     }
   }
 
+  /**
+   * The default {@link CompletionCallback}. The default completion callback is used to complete
+   * transform evaluations that are triggered due to the arrival of elements from an upstream
+   * transform, or for a source transform.
+   */
+  private class DefaultCompletionCallback extends CompletionCallbackBase {
+    @Override
+    public CommittedResult getCommittedResult(
+        CommittedBundle inputBundle, InProcessTransformResult result) {
+      return evaluationContext.handleResult(inputBundle,
+          Collections.emptyList(),
+          result);
+    }
+  }
+
   /**
    * A {@link CompletionCallback} where the completed bundle was produced to deliver some collection
    * of {@link TimerData timers}. When the evaluator completes successfully, reports all of the
    * timers used to create the input to the {@link InProcessEvaluationContext evaluation context}
    * as part of the result.
    */
-  private class TimerCompletionCallback implements CompletionCallback {
+  private class TimerCompletionCallback extends CompletionCallbackBase {
     private final Iterable timers;
 
     private TimerCompletionCallback(Iterable timers) {
@@ -246,19 +265,9 @@ private TimerCompletionCallback(Iterable timers) {
     }
 
     @Override
-    public CommittedResult handleResult(
+    public CommittedResult getCommittedResult(
         CommittedBundle inputBundle, InProcessTransformResult result) {
-      CommittedResult committedResult =
-          evaluationContext.handleResult(inputBundle, timers, result);
-      for (CommittedBundle outputBundle : committedResult.getOutputs()) {
-        allUpdates.offer(ExecutorUpdate.fromBundle(outputBundle));
-      }
-      return committedResult;
-    }
-
-    @Override
-    public void handleThrowable(CommittedBundle inputBundle, Throwable t) {
-      allUpdates.offer(ExecutorUpdate.fromThrowable(t));
+          return evaluationContext.handleResult(inputBundle, timers, result);
     }
   }
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java
index 874ec1780bc5..9a08996be215 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactory.java
@@ -19,6 +19,7 @@
 
 import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
 
+import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
 import org.apache.beam.runners.direct.InProcessPipelineRunner.CommittedBundle;
 import org.apache.beam.runners.direct.InProcessPipelineRunner.UncommittedBundle;
 import org.apache.beam.runners.direct.StepTransformResult.Builder;
@@ -32,7 +33,6 @@
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn;
 import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.ReifyTimestampsAndWindows;
 import org.apache.beam.sdk.util.KeyedWorkItem;
 import org.apache.beam.sdk.util.KeyedWorkItemCoder;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
index 2103ad3b2b91..3b3821199beb 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
@@ -31,7 +31,6 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 
-import com.google.api.client.util.Throwables;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.SetMultimap;
 
@@ -100,7 +99,7 @@ public UncommittedBundle add(WindowedValue element) {
         mutationDetectors.put(
             element, MutationDetectors.forValueWithCoder(element.getValue(), coder));
       } catch (CoderException e) {
-        throw Throwables.propagate(e);
+        throw new RuntimeException(e);
       }
       underlying.add(element);
       return this;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java
index 769457a77a8c..4d5a3a13f605 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InMemoryWatermarkManager.java
@@ -800,18 +800,19 @@ public TransformWatermarks getWatermarks(AppliedPTransform transform) {
    * 
. * * @param completed the input that has completed - * @param transform the transform that has completed processing the input - * @param outputs the bundles the transform has output + * @param timerUpdate the timers that were added, removed, and completed as part of producing + * this update + * @param result the result that was produced by processing the input * @param earliestHold the earliest watermark hold in the transform's state. {@code null} if there * is no hold */ public void updateWatermarks( @Nullable CommittedBundle completed, - AppliedPTransform transform, TimerUpdate timerUpdate, - Iterable> outputs, + CommittedResult result, @Nullable Instant earliestHold) { - updatePending(completed, transform, timerUpdate, outputs); + AppliedPTransform transform = result.getTransform(); + updatePending(completed, timerUpdate, result); TransformWatermarks transformWms = transformToWatermarks.get(transform); transformWms.setEventTimeHold(completed == null ? null : completed.getKey(), earliestHold); refreshWatermarks(transform); @@ -846,15 +847,14 @@ private void refreshWatermarks(AppliedPTransform transform) { */ private void updatePending( CommittedBundle input, - AppliedPTransform transform, TimerUpdate timerUpdate, - Iterable> outputs) { - TransformWatermarks completedTransform = transformToWatermarks.get(transform); + CommittedResult result) { + TransformWatermarks completedTransform = transformToWatermarks.get(result.getTransform()); // Newly pending elements must be added before completed elements are removed, as the two // do not share a Mutex within this call and thus can be interleaved with external calls to // refresh. - for (CommittedBundle bundle : outputs) { + for (CommittedBundle bundle : result.getOutputs()) { for (AppliedPTransform consumer : consumers.get(bundle.getPCollection())) { TransformWatermarks watermarks = transformToWatermarks.get(consumer); watermarks.addPending(bundle); @@ -865,7 +865,6 @@ private void updatePending( if (input != null) { completedTransform.removePending(input); } - } /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java index d9a7ff02f839..bcf85da9d52d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessEvaluationContext.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.util.ExecutionContext; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowedValue; @@ -152,11 +153,11 @@ public synchronized CommittedResult handleResult( Iterable> committedBundles = commitBundles(result.getOutputBundles()); // Update watermarks and timers + CommittedResult committedResult = CommittedResult.create(result, committedBundles); watermarkManager.updateWatermarks( completedBundle, - result.getTransform(), result.getTimerUpdate().withCompletedTimers(completedTimers), - committedBundles, + committedResult, result.getWatermarkHold()); fireAllAvailableCallbacks(); // Update counters @@ -176,7 +177,7 @@ public synchronized CommittedResult handleResult( applicationStateInternals.remove(stepAndKey); } } - return CommittedResult.create(result, committedBundles); + return committedResult; } private Iterable> commitBundles( @@ -333,16 +334,6 @@ public ReadyCheckingSideInputReader createSideInputReader( return sideInputContainer.createReaderForViews(sideInputs); } - /** - * A {@link SideInputReader} that allows callers to check to see if a {@link PCollectionView} has - * had its contents set in a window. - */ - static interface ReadyCheckingSideInputReader extends SideInputReader { - /** - * Returns true if the {@link PCollectionView} is ready in the provided {@link BoundedWindow}. - */ - boolean isReady(PCollectionView view, BoundedWindow window); - } /** * Create a {@link CounterSet} for this {@link Pipeline}. The {@link CounterSet} is independent diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java index bb8c0de68613..19e9f47de568 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessPipelineRunner.java @@ -50,7 +50,6 @@ import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -256,7 +255,10 @@ public InProcessPipelineResult run(Pipeline pipeline) { } catch (UserCodeException userException) { throw new PipelineExecutionException(userException.getCause()); } catch (Throwable t) { - Throwables.propagate(t); + if (t instanceof RuntimeException) { + throw (RuntimeException) t; + } + throw new RuntimeException(t); } } return result; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java index f4980ef1546c..78889dc69689 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/InProcessSideInputContainer.java @@ -19,17 +19,16 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.runners.direct.InProcessEvaluationContext.ReadyCheckingSideInputReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.PCollectionViewWindow; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import com.google.common.base.MoreObjects; -import com.google.common.base.Throwables; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; @@ -172,7 +171,7 @@ private void updatePCollectionViewWindowValues( future.set(Collections.>emptyList()); } } catch (ExecutionException e) { - Throwables.propagate(e.getCause()); + throw new RuntimeException(e.getCause()); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java index 8346e89f5af4..9e15c2aab830 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java @@ -23,8 +23,6 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.util.WindowedValue; -import com.google.common.base.Throwables; - import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.Callable; @@ -119,7 +117,10 @@ public InProcessTransformResult call() { return result; } catch (Throwable t) { onComplete.handleThrowable(inputBundle, t); - throw Throwables.propagate(t); + if (t instanceof RuntimeException) { + throw (RuntimeException) t; + } + throw new RuntimeException(t); } finally { transformEvaluationState.complete(this); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java index 2880adec6832..15cdf8a32e06 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InMemoryWatermarkManagerTest.java @@ -159,8 +159,11 @@ public void getWatermarkForUntouchedTransform() { @Test public void getWatermarkForUpdatedSourceTransform() { CommittedBundle output = multiWindowedBundle(createdInts, 1); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(output), new Instant(8000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(output)), + new Instant(8000L)); TransformWatermarks updatedSourceWatermark = manager.getWatermarks(createdInts.getProducingTransformInternal()); @@ -175,8 +178,10 @@ public void getWatermarkForUpdatedSourceTransform() { public void getWatermarkForMultiInputTransform() { CommittedBundle secondPcollectionBundle = multiWindowedBundle(intsToFlatten, -1); - manager.updateWatermarks(null, intsToFlatten.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(secondPcollectionBundle), + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(intsToFlatten.getProducingTransformInternal(), + Collections.>singleton(secondPcollectionBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); // We didn't do anything for the first source, so we shouldn't have progressed the watermark @@ -205,13 +210,17 @@ public void getWatermarkForMultiInputTransform() { CommittedBundle flattenedBundleSecondCreate = multiWindowedBundle(flattened, -1); // We have finished processing the bundle from the second PCollection, but we haven't consumed // anything from the first PCollection yet; so our watermark shouldn't advance - manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(flattenedBundleSecondCreate), + manager.updateWatermarks(secondPcollectionBundle, + TimerUpdate.empty(), + result(flattened.getProducingTransformInternal(), + Collections.>singleton(flattenedBundleSecondCreate)), null); TransformWatermarks transformAfterProcessing = manager.getWatermarks(flattened.getProducingTransformInternal()); - manager.updateWatermarks(secondPcollectionBundle, flattened.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(flattenedBundleSecondCreate), + manager.updateWatermarks(secondPcollectionBundle, + TimerUpdate.empty(), + result(flattened.getProducingTransformInternal(), + Collections.>singleton(flattenedBundleSecondCreate)), null); assertThat( transformAfterProcessing.getInputWatermark(), @@ -225,8 +234,10 @@ public void getWatermarkForMultiInputTransform() { timestampedBundle(createdInts, TimestampedValue.of(5, firstCollectionTimestamp)); // the source is done, but elements are still buffered. The source output watermark should be // past the end of the global window - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(firstPcollectionBundle), + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(firstPcollectionBundle)), new Instant(Long.MAX_VALUE)); TransformWatermarks firstSourceWatermarks = manager.getWatermarks(createdInts.getProducingTransformInternal()); @@ -253,8 +264,10 @@ public void getWatermarkForMultiInputTransform() { CommittedBundle completedFlattenBundle = bundleFactory.createRootBundle(flattened).commit(BoundedWindow.TIMESTAMP_MAX_VALUE); - manager.updateWatermarks(firstPcollectionBundle, flattened.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(completedFlattenBundle), + manager.updateWatermarks(firstPcollectionBundle, + TimerUpdate.empty(), + result(flattened.getProducingTransformInternal(), + Collections.>singleton(completedFlattenBundle)), null); TransformWatermarks afterConsumingAllInput = manager.getWatermarks(flattened.getProducingTransformInternal()); @@ -275,8 +288,11 @@ public void getWatermarkForMultiConsumedCollection() { CommittedBundle createdBundle = timestampedBundle(createdInts, TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)), TimestampedValue.of(3, new Instant(-1000L))); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createdBundle), new Instant(Long.MAX_VALUE)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createdBundle)), + new Instant(Long.MAX_VALUE)); TransformWatermarks createdAfterProducing = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat( @@ -287,8 +303,11 @@ public void getWatermarkForMultiConsumedCollection() { timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)), TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)), TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); - manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(keyBundle), null); + manager.updateWatermarks(createdBundle, + TimerUpdate.empty(), + result(keyed.getProducingTransformInternal(), + Collections.>singleton(keyBundle)), + null); TransformWatermarks keyedWatermarks = manager.getWatermarks(keyed.getProducingTransformInternal()); assertThat( @@ -303,8 +322,11 @@ public void getWatermarkForMultiConsumedCollection() { CommittedBundle filteredBundle = timestampedBundle(filtered, TimestampedValue.of(2, new Instant(1234L))); - manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(filteredBundle), null); + manager.updateWatermarks(createdBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filteredBundle)), + null); TransformWatermarks filteredProcessedWatermarks = manager.getWatermarks(filtered.getProducingTransformInternal()); assertThat( @@ -322,17 +344,23 @@ public void getWatermarkForMultiConsumedCollection() { @Test public void updateWatermarkWithWatermarkHolds() { CommittedBundle createdBundle = timestampedBundle(createdInts, - TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)), + TimestampedValue.of(1, new Instant(1_000_000L)), + TimestampedValue.of(2, new Instant(1234L)), TimestampedValue.of(3, new Instant(-1000L))); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createdBundle), new Instant(Long.MAX_VALUE)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createdBundle)), + new Instant(Long.MAX_VALUE)); - CommittedBundle> keyBundle = - timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)), - TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)), - TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); - manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(keyBundle), + CommittedBundle> keyBundle = timestampedBundle(keyed, + TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)), + TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)), + TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); + manager.updateWatermarks(createdBundle, + TimerUpdate.empty(), + result(keyed.getProducingTransformInternal(), + Collections.>singleton(keyBundle)), new Instant(500L)); TransformWatermarks keyedWatermarks = manager.getWatermarks(keyed.getProducingTransformInternal()); @@ -358,40 +386,54 @@ public void updateWatermarkWithKeyedWatermarkHolds() { .add(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1234L))) .commit(clock.now()); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - ImmutableList.of(firstKeyBundle, secondKeyBundle), BoundedWindow.TIMESTAMP_MAX_VALUE); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + ImmutableList.of(firstKeyBundle, secondKeyBundle)), + BoundedWindow.TIMESTAMP_MAX_VALUE); - manager.updateWatermarks(firstKeyBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), new Instant(-1000L)); - manager.updateWatermarks(secondKeyBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), new Instant(1234L)); + manager.updateWatermarks(firstKeyBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(-1000L)); + manager.updateWatermarks(secondKeyBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(1234L)); TransformWatermarks filteredWatermarks = manager.getWatermarks(filtered.getProducingTransformInternal()); - assertThat( - filteredWatermarks.getInputWatermark(), + assertThat(filteredWatermarks.getInputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); assertThat(filteredWatermarks.getOutputWatermark(), not(laterThan(new Instant(-1000L)))); CommittedBundle fauxFirstKeyTimerBundle = bundleFactory.createKeyedBundle(null, "Odd", createdInts).commit(clock.now()); - manager.updateWatermarks(fauxFirstKeyTimerBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), + manager.updateWatermarks(fauxFirstKeyTimerBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(1234L))); CommittedBundle fauxSecondKeyTimerBundle = bundleFactory.createKeyedBundle(null, "Even", createdInts).commit(clock.now()); - manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), new Instant(5678L)); + manager.updateWatermarks(fauxSecondKeyTimerBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(5678L)); assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(5678L))); - manager.updateWatermarks(fauxSecondKeyTimerBundle, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), + manager.updateWatermarks(fauxSecondKeyTimerBundle, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); - assertThat( - filteredWatermarks.getOutputWatermark(), + assertThat(filteredWatermarks.getOutputWatermark(), not(earlierThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); } @@ -403,16 +445,21 @@ public void updateWatermarkWithKeyedWatermarkHolds() { public void updateOutputWatermarkShouldBeMonotonic() { CommittedBundle firstInput = bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(firstInput), new Instant(0L)); + manager.updateWatermarks(null, TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(firstInput)), + new Instant(0L)); TransformWatermarks firstWatermarks = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat(firstWatermarks.getOutputWatermark(), equalTo(new Instant(0L))); CommittedBundle secondInput = bundleFactory.createRootBundle(createdInts).commit(BoundedWindow.TIMESTAMP_MAX_VALUE); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(secondInput), new Instant(-250L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(secondInput)), + new Instant(-250L)); TransformWatermarks secondWatermarks = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat(secondWatermarks.getOutputWatermark(), not(earlierThan(new Instant(0L)))); @@ -425,17 +472,22 @@ public void updateOutputWatermarkShouldBeMonotonic() { @Test public void updateWatermarkWithHoldsShouldBeMonotonic() { CommittedBundle createdBundle = timestampedBundle(createdInts, - TimestampedValue.of(1, new Instant(1_000_000L)), TimestampedValue.of(2, new Instant(1234L)), - TimestampedValue.of(3, new Instant(-1000L))); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createdBundle), new Instant(Long.MAX_VALUE)); + TimestampedValue.of(1, new Instant(1_000_000L)), + TimestampedValue.of(2, new Instant(1234L)), + TimestampedValue.of(3, new Instant(-1000L))); manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createdBundle)), + new Instant(Long.MAX_VALUE)); CommittedBundle> keyBundle = timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), new Instant(1_000_000L)), TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L)), TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); - manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(keyBundle), + manager.updateWatermarks(createdBundle, + TimerUpdate.empty(), + result(keyed.getProducingTransformInternal(), + Collections.>singleton(keyBundle)), new Instant(500L)); TransformWatermarks keyedWatermarks = manager.getWatermarks(keyed.getProducingTransformInternal()); @@ -462,16 +514,22 @@ public void updateWatermarkWithLateData() { CommittedBundle createdBundle = timestampedBundle(createdInts, TimestampedValue.of(1, sourceWatermark), TimestampedValue.of(2, new Instant(1234L))); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createdBundle), sourceWatermark); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createdBundle)), + sourceWatermark); CommittedBundle> keyBundle = timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 1), sourceWatermark), TimestampedValue.of(KV.of("MyKey", 2), new Instant(1234L))); // Finish processing the on-time data. The watermarks should progress to be equal to the source - manager.updateWatermarks(createdBundle, keyed.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(keyBundle), null); + manager.updateWatermarks(createdBundle, + TimerUpdate.empty(), + result(keyed.getProducingTransformInternal(), + Collections.>singleton(keyBundle)), + null); TransformWatermarks onTimeWatermarks = manager.getWatermarks(keyed.getProducingTransformInternal()); assertThat(onTimeWatermarks.getInputWatermark(), equalTo(sourceWatermark)); @@ -481,8 +539,11 @@ public void updateWatermarkWithLateData() { timestampedBundle(createdInts, TimestampedValue.of(3, new Instant(-1000L))); // the late data arrives in a downstream PCollection after its watermark has advanced past it; // we don't advance the watermark past the current watermark until we've consumed the late data - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(lateDataBundle), new Instant(2_000_000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(lateDataBundle)), + new Instant(2_000_000L)); TransformWatermarks bufferedLateWm = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat(bufferedLateWm.getOutputWatermark(), equalTo(new Instant(2_000_000L))); @@ -496,30 +557,31 @@ public void updateWatermarkWithLateData() { CommittedBundle> lateKeyedBundle = timestampedBundle(keyed, TimestampedValue.of(KV.of("MyKey", 3), new Instant(-1000L))); - manager.updateWatermarks(lateDataBundle, keyed.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(lateKeyedBundle), null); + manager.updateWatermarks(lateDataBundle, + TimerUpdate.empty(), + result(keyed.getProducingTransformInternal(), + Collections.>singleton(lateKeyedBundle)), + null); } public void updateWatermarkWithDifferentWindowedValueInstances() { manager.updateWatermarks( null, - createdInts.getProducingTransformInternal(), TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), Collections.>singleton( bundleFactory .createRootBundle(createdInts) .add(WindowedValue.valueInGlobalWindow(1)) - .commit(Instant.now())), + .commit(Instant.now()))), BoundedWindow.TIMESTAMP_MAX_VALUE); manager.updateWatermarks( - bundleFactory - .createRootBundle(createdInts) + bundleFactory.createRootBundle(createdInts) .add(WindowedValue.valueInGlobalWindow(1)) .commit(Instant.now()), - keyed.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>emptyList(), + result(keyed.getProducingTransformInternal(), Collections.>emptyList()), null); TransformWatermarks onTimeWatermarks = manager.getWatermarks(keyed.getProducingTransformInternal()); @@ -533,8 +595,10 @@ public void updateWatermarkWithDifferentWindowedValueInstances() { @Test public void getWatermarksAfterOnlyEmptyOutput() { CommittedBundle emptyCreateOutput = multiWindowedBundle(createdInts); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(emptyCreateOutput), + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(emptyCreateOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks updatedSourceWatermarks = manager.getWatermarks(createdInts.getProducingTransformInternal()); @@ -560,12 +624,17 @@ public void getWatermarksAfterOnlyEmptyOutput() { @Test public void getWatermarksAfterHoldAndEmptyOutput() { CommittedBundle firstCreateOutput = multiWindowedBundle(createdInts, 1, 2); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(firstCreateOutput), new Instant(12_000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(firstCreateOutput)), + new Instant(12_000L)); CommittedBundle firstFilterOutput = multiWindowedBundle(filtered); - manager.updateWatermarks(firstCreateOutput, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(firstFilterOutput), + manager.updateWatermarks(firstCreateOutput, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(firstFilterOutput)), new Instant(10_000L)); TransformWatermarks firstFilterWatermarks = manager.getWatermarks(filtered.getProducingTransformInternal()); @@ -573,8 +642,10 @@ public void getWatermarksAfterHoldAndEmptyOutput() { assertThat(firstFilterWatermarks.getOutputWatermark(), not(laterThan(new Instant(10_000L)))); CommittedBundle emptyCreateOutput = multiWindowedBundle(createdInts); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(emptyCreateOutput), + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(emptyCreateOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks updatedSourceWatermarks = manager.getWatermarks(createdInts.getProducingTransformInternal()); @@ -613,8 +684,11 @@ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { CommittedBundle createOutput = bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createOutput)), + BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks createAfterUpdate = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now())); @@ -639,8 +713,10 @@ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { CommittedBundle filterOutputBundle = bundleFactory.createRootBundle(intsToFlatten).commit(new Instant(1250L)); - manager.updateWatermarks(createOutput, filtered.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>singleton(filterOutputBundle), + manager.updateWatermarks(createOutput, + TimerUpdate.empty(), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filterOutputBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks filterAfterConsumed = manager.getWatermarks(filtered.getProducingTransformInternal()); @@ -661,8 +737,10 @@ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { // @Test public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { CommittedBundle createdBundle = multiWindowedBundle(createdInts, 1, 2, 4, 8); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createdBundle), new Instant(1248L)); + manager.updateWatermarks(null, TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createdBundle)), + new Instant(1248L)); TransformWatermarks filteredWms = manager.getWatermarks(filtered.getProducingTransformInternal()); @@ -678,8 +756,10 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { TimerData.of(StateNamespaces.global(), new Instant(4096L), TimeDomain.PROCESSING_TIME); TimerUpdate timers = TimerUpdate.builder("key").setTimer(pastTimer).setTimer(futureTimer).build(); - manager.updateWatermarks(createdBundle, filtered.getProducingTransformInternal(), timers, - Collections.>singleton(filteredBundle), + manager.updateWatermarks(createdBundle, + timers, + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filteredBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); Instant startTime = clock.now(); clock.set(startTime.plus(250L)); @@ -712,11 +792,11 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { bundleFactory.createKeyedBundle(null, "key", filteredTimesTwo) .commit(filteredWms.getSynchronizedProcessingOutputTime()); // Complete the processing time timer - manager.updateWatermarks(filteredTimerBundle, filtered.getProducingTransformInternal(), + manager.updateWatermarks(filteredTimerBundle, TimerUpdate.builder("key") - .withCompletedTimers(Collections.singleton(pastTimer)) - .build(), - Collections.>singleton(filteredTimerResult), + .withCompletedTimers(Collections.singleton(pastTimer)).build(), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filteredTimerResult)), BoundedWindow.TIMESTAMP_MAX_VALUE); clock.set(startTime.plus(500L)); @@ -726,8 +806,10 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { filteredDoubledWms.getSynchronizedProcessingOutputTime(), not(earlierThan(filteredTimerResult.getSynchronizedProcessingOutputWatermark()))); - manager.updateWatermarks(filteredTimerResult, filteredTimesTwo.getProducingTransformInternal(), - TimerUpdate.empty(), Collections.>emptyList(), + manager.updateWatermarks(filteredTimerResult, + TimerUpdate.empty(), + result(filteredTimesTwo.getProducingTransformInternal(), + Collections.>emptyList()), BoundedWindow.TIMESTAMP_MAX_VALUE); assertThat(filteredDoubledWms.getSynchronizedProcessingOutputTime(), equalTo(clock.now())); @@ -761,18 +843,23 @@ public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() { CommittedBundle createOutput = bundleFactory.createRootBundle(createdInts).commit(new Instant(1250L)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createOutput), BoundedWindow.TIMESTAMP_MAX_VALUE); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createOutput)), + BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks createAfterUpdate = manager.getWatermarks(createdInts.getProducingTransformInternal()); assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), not(laterThan(clock.now()))); - assertThat( - createAfterUpdate.getSynchronizedProcessingOutputTime(), not(laterThan(clock.now()))); + assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), + not(laterThan(clock.now()))); CommittedBundle createSecondOutput = bundleFactory.createRootBundle(createdInts).commit(new Instant(750L)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(createSecondOutput), + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(createSecondOutput)), BoundedWindow.TIMESTAMP_MAX_VALUE); assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now())); @@ -781,16 +868,20 @@ public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() { @Test public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers() { CommittedBundle created = multiWindowedBundle(createdInts, 1, 2, 3); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(created), new Instant(40_900L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(created)), + new Instant(40_900L)); CommittedBundle filteredBundle = multiWindowedBundle(filtered, 2, 4); Instant upstreamHold = new Instant(2048L); TimerData upstreamProcessingTimer = TimerData.of(StateNamespaces.global(), upstreamHold, TimeDomain.PROCESSING_TIME); - manager.updateWatermarks(created, filtered.getProducingTransformInternal(), + manager.updateWatermarks(created, TimerUpdate.builder("key").setTimer(upstreamProcessingTimer).build(), - Collections.>singleton(filteredBundle), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filteredBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks downstreamWms = @@ -806,11 +897,12 @@ public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers( assertThat(downstreamWms.getSynchronizedProcessingInputTime(), equalTo(upstreamHold)); CommittedBundle otherCreated = multiWindowedBundle(createdInts, 4, 8, 12); - manager.updateWatermarks(otherCreated, filtered.getProducingTransformInternal(), + manager.updateWatermarks(otherCreated, TimerUpdate.builder("key") - .withCompletedTimers(Collections.singleton(upstreamProcessingTimer)) - .build(), - Collections.>emptyList(), BoundedWindow.TIMESTAMP_MAX_VALUE); + .withCompletedTimers(Collections.singleton(upstreamProcessingTimer)).build(), + result(filtered.getProducingTransformInternal(), + Collections.>emptyList()), + BoundedWindow.TIMESTAMP_MAX_VALUE); assertThat(downstreamWms.getSynchronizedProcessingInputTime(), not(earlierThan(clock.now()))); } @@ -820,9 +912,9 @@ public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() { CommittedBundle created = multiWindowedBundle(createdInts, 1, 2, 3); manager.updateWatermarks( null, - createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(created), + result(createdInts.getProducingTransformInternal(), + Collections.>singleton(created)), new Instant(29_919_235L)); Instant upstreamHold = new Instant(2048L); @@ -830,9 +922,9 @@ public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() { bundleFactory.createKeyedBundle(created, "key", filtered).commit(upstreamHold); manager.updateWatermarks( created, - filtered.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>singleton(filteredBundle), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(filteredBundle)), BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks downstreamWms = @@ -852,8 +944,10 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { // Advance WM of keyed past the first timer, but ahead of the second and third CommittedBundle createdBundle = multiWindowedBundle(filtered); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.singleton(createdBundle), new Instant(1500L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), Collections.singleton(createdBundle)), + new Instant(1500L)); TimerData earliestTimer = TimerData.of(StateNamespaces.global(), new Instant(1000), TimeDomain.EVENT_TIME); @@ -869,11 +963,10 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { .setTimer(lastTimer) .build(); - manager.updateWatermarks( - createdBundle, - filtered.getProducingTransformInternal(), + manager.updateWatermarks(createdBundle, update, - Collections.>singleton(multiWindowedBundle(intsToFlatten)), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(multiWindowedBundle(intsToFlatten))), new Instant(1000L)); Map, Map> firstTransformFiredTimers = @@ -886,8 +979,11 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { FiredTimers firstFired = firstFilteredTimers.get(key); assertThat(firstFired.getTimers(TimeDomain.EVENT_TIME), contains(earliestTimer)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>emptyList(), new Instant(50_000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(50_000L)); Map, Map> secondTransformFiredTimers = manager.extractFiredTimers(); assertThat( @@ -909,8 +1005,10 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { // Advance WM of keyed past the first timer, but ahead of the second and third CommittedBundle createdBundle = multiWindowedBundle(filtered); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.singleton(createdBundle), new Instant(1500L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), Collections.singleton(createdBundle)), + new Instant(1500L)); TimerData earliestTimer = TimerData.of(StateNamespaces.global(), new Instant(999L), TimeDomain.PROCESSING_TIME); @@ -928,9 +1026,9 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { manager.updateWatermarks( createdBundle, - filtered.getProducingTransformInternal(), update, - Collections.>singleton(multiWindowedBundle(intsToFlatten)), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(multiWindowedBundle(intsToFlatten))), new Instant(1000L)); Map, Map> firstTransformFiredTimers = @@ -944,8 +1042,11 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { assertThat(firstFired.getTimers(TimeDomain.PROCESSING_TIME), contains(earliestTimer)); clock.set(new Instant(50_000L)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>emptyList(), new Instant(50_000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(50_000L)); Map, Map> secondTransformFiredTimers = manager.extractFiredTimers(); assertThat( @@ -967,8 +1068,10 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { // Advance WM of keyed past the first timer, but ahead of the second and third CommittedBundle createdBundle = multiWindowedBundle(filtered); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.singleton(createdBundle), new Instant(1500L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), Collections.singleton(createdBundle)), + new Instant(1500L)); TimerData earliestTimer = TimerData.of( StateNamespaces.global(), new Instant(999L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME); @@ -986,9 +1089,9 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { manager.updateWatermarks( createdBundle, - filtered.getProducingTransformInternal(), update, - Collections.>singleton(multiWindowedBundle(intsToFlatten)), + result(filtered.getProducingTransformInternal(), + Collections.>singleton(multiWindowedBundle(intsToFlatten))), new Instant(1000L)); Map, Map> firstTransformFiredTimers = @@ -1003,8 +1106,11 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { firstFired.getTimers(TimeDomain.SYNCHRONIZED_PROCESSING_TIME), contains(earliestTimer)); clock.set(new Instant(50_000L)); - manager.updateWatermarks(null, createdInts.getProducingTransformInternal(), TimerUpdate.empty(), - Collections.>emptyList(), new Instant(50_000L)); + manager.updateWatermarks(null, + TimerUpdate.empty(), + result(createdInts.getProducingTransformInternal(), + Collections.>emptyList()), + new Instant(50_000L)); Map, Map> secondTransformFiredTimers = manager.extractFiredTimers(); assertThat( @@ -1133,7 +1239,6 @@ public boolean matches(Object item) { ReadableInstant instant = (ReadableInstant) item; return instant.isAfter(shouldBeEarlier); } - @Override public void describeTo(Description description) { description.appendText("later than ").appendValue(shouldBeEarlier); @@ -1165,4 +1270,11 @@ private final CommittedBundle multiWindowedBundle(PCollection pc, T... } return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE); } + + private final CommittedResult result( + AppliedPTransform transform, + Iterable> bundles) { + return CommittedResult.create(StepTransformResult.withoutHold(transform) + .build(), bundles); + } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java index d8a78f23d3e8..8f89e707b9bb 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/InProcessSideInputContainerTest.java @@ -24,7 +24,6 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.doAnswer; -import org.apache.beam.runners.direct.InProcessEvaluationContext.ReadyCheckingSideInputReader; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; @@ -37,6 +36,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.util.PCollectionViews; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml index d27f2db2f361..e5bab3e9c03e 100644 --- a/runners/flink/examples/pom.xml +++ b/runners/flink/examples/pom.xml @@ -79,11 +79,16 @@ + org.codehaus.mojo exec-maven-plugin - 1.2.1 java diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index 608dc7c0020a..5e05e6a09f20 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -44,7 +44,6 @@ UTF-8 UTF-8 1.0.0 - 0.1.0-incubating-SNAPSHOT @@ -62,16 +61,27 @@ + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-source-plugin + + - - org.apache.maven.plugins maven-failsafe-plugin - 2.17 @@ -90,7 +100,6 @@ org.apache.maven.plugins maven-surefire-plugin - 2.17 -Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit @@ -99,60 +108,15 @@ org.codehaus.mojo exec-maven-plugin - 1.2.1 none - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - 1.7 - 1.7 - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - 1.3.1 - - - enforce-maven - - enforce - - - - - [1.7,) - - - - [3.0.3,) - - - - - - - - - - diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 89c8d9949e66..18e81d259b74 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -34,31 +34,6 @@ jar - - - disable-runnable-on-service-tests - - true - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - runnable-on-service-tests - - true - - - - - - - - - @@ -87,11 +62,22 @@ flink-avro_2.10 ${flink.version} + org.apache.beam java-sdk-all - ${beam.version} + + + org.slf4j + slf4j-jdk14 + + + + + + org.apache.beam + runners-core org.slf4j @@ -103,7 +89,6 @@ org.apache.beam google-cloud-dataflow-java-runner - ${project.version} org.slf4j @@ -111,11 +96,42 @@ + + + + org.hamcrest + hamcrest-all + test + + + junit + junit + test + + + org.mockito + mockito-all + test + + + + + org.apache.beam + java-sdk-all + tests + test + + + org.slf4j + slf4j-jdk14 + + + + org.apache.beam java-examples-all - ${beam.version} org.slf4j @@ -135,19 +151,11 @@ org.apache.flink flink-test-utils_2.10 ${flink.version} - test - - - org.mockito - mockito-all - 1.9.5 - test com.google.auto.service auto-service - 1.0-rc2 true @@ -155,6 +163,13 @@ + + org.apache.maven.plugins @@ -165,10 +180,60 @@ org.apache.maven.plugins maven-surefire-plugin + + + runnable-on-service-tests + integration-test + + test + + + org.apache.beam.sdk.testing.RunnableOnService + none + true + + org.apache.beam:java-sdk-all + + + + [ + "--runner=org.apache.beam.runners.flink.TestFlinkPipelineRunner", + "--streaming=false" + ] + + + + + + streaming-runnable-on-service-tests + integration-test + + test + + + true + org.apache.beam.sdk.testing.RunnableOnService + none + true + + org.apache.beam:java-sdk-all + + + + [ + "--runner=org.apache.beam.runners.flink.TestFlinkPipelineRunner", + "--streaming=true" + ] + + + + + + + - diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java index a389d7a076c8..5e44d997e970 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java @@ -110,7 +110,7 @@ public FlinkRunnerResult run(Pipeline pipeline) { this.flinkJobEnv.translate(pipeline); LOG.info("Starting execution of Flink program."); - + JobExecutionResult result; try { result = this.flinkJobEnv.executePipeline(); @@ -140,20 +140,6 @@ 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) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java index cd99f4e65bce..ec61805a4ed0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java @@ -41,7 +41,9 @@ private FlinkRunnerRegistrar() { } public static class Runner implements PipelineRunnerRegistrar { @Override public Iterable>> getPipelineRunners() { - return ImmutableList.>>of(FlinkPipelineRunner.class); + return ImmutableList.>>of( + FlinkPipelineRunner.class, + TestFlinkPipelineRunner.class); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java new file mode 100644 index 000000000000..139aebf9dd2b --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkPipelineRunner.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.PipelineOptionsValidator; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; + +import org.apache.flink.runtime.client.JobExecutionException; + +public class TestFlinkPipelineRunner extends PipelineRunner { + + private FlinkPipelineRunner delegate; + + private TestFlinkPipelineRunner(FlinkPipelineOptions options) { + // We use [auto] for testing since this will make it pick up the Testing ExecutionEnvironment + options.setFlinkMaster("[auto]"); + this.delegate = FlinkPipelineRunner.fromOptions(options); + } + + public static TestFlinkPipelineRunner fromOptions(PipelineOptions options) { + FlinkPipelineOptions flinkOptions = PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options); + return new TestFlinkPipelineRunner(flinkOptions); + } + + public static TestFlinkPipelineRunner create(boolean streaming) { + FlinkPipelineOptions flinkOptions = PipelineOptionsFactory.as(FlinkPipelineOptions.class); + flinkOptions.setStreaming(streaming); + return TestFlinkPipelineRunner.fromOptions(flinkOptions); + } + + @Override + public + OutputT apply(PTransform transform, InputT input) { + return delegate.apply(transform, input); + } + + @Override + public FlinkRunnerResult run(Pipeline pipeline) { + try { + return delegate.run(pipeline); + } catch (RuntimeException e) { + // Special case hack to pull out assertion errors from PAssert; instead there should + // probably be a better story along the lines of UserCodeException. + if (e.getCause() != null + && e.getCause() instanceof JobExecutionException + && e.getCause().getCause() instanceof AssertionError) { + throw (AssertionError) e.getCause().getCause(); + } else { + throw e; + } + } + } + + public PipelineOptions getPipelineOptions() { + return delegate.getPipelineOptions(); + } +} + + diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java index 456cf09457b6..98229ae331f3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.translation; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -24,13 +25,15 @@ import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.values.PValue; +import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * FlinkBatchPipelineTranslator knows how to translate Pipeline objects into Flink Jobs. - * This is based on {@link org.apache.beam.runners.dataflow.DataflowPipelineTranslator} + * {@link Pipeline.PipelineVisitor} for executing a {@link Pipeline} as a + * Flink batch job. */ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator { @@ -52,6 +55,17 @@ public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions op this.batchContext = new FlinkBatchTranslationContext(env, options); } + @Override + @SuppressWarnings("rawtypes, unchecked") + public void translate(Pipeline pipeline) { + super.translate(pipeline); + + // terminate dangling DataSets + for (DataSet dataSet: batchContext.getDanglingDataSets().values()) { + dataSet.output(new DiscardingOutputFormat()); + } + } + // -------------------------------------------------------------------------------------------- // Pipeline Visitor Methods // -------------------------------------------------------------------------------------------- @@ -63,10 +77,13 @@ public void enterCompositeTransform(TransformTreeNode node) { PTransform transform = node.getTransform(); if (transform != null && currentCompositeTransform == null) { - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); + if (translator != null) { currentCompositeTransform = transform; - if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) { + if (transform instanceof CoGroupByKey && + node.getInput().expand().size() != 2) { // we can only optimize CoGroupByKey for input size 2 currentCompositeTransform = null; } @@ -80,7 +97,9 @@ public void leaveCompositeTransform(TransformTreeNode node) { PTransform transform = node.getTransform(); if (transform != null && currentCompositeTransform == transform) { - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); + if (translator != null) { LOG.info(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node)); applyBatchTransform(transform, node, translator); @@ -106,10 +125,13 @@ public void visitTransform(TransformTreeNode node) { // currently visiting and translate it into its Flink alternative. PTransform transform = node.getTransform(); - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); + if (translator == null) { LOG.info(node.getTransform().getClass().toString()); - throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); + throw new UnsupportedOperationException( + "The transform " + transform + " is currently not supported."); } applyBatchTransform(transform, node, translator); } @@ -119,7 +141,10 @@ public void visitValue(PValue value, TransformTreeNode producer) { // do nothing here } - private > void applyBatchTransform(PTransform transform, TransformTreeNode node, BatchTransformTranslator translator) { + private > void applyBatchTransform( + PTransform transform, + TransformTreeNode node, + BatchTransformTranslator translator) { @SuppressWarnings("unchecked") T typedTransform = (T) transform; @@ -136,8 +161,8 @@ public void visitValue(PValue value, TransformTreeNode producer) { /** * A translator of a {@link PTransform}. */ - public interface BatchTransformTranslator { - void translateNode(Type transform, FlinkBatchTranslationContext context); + public interface BatchTransformTranslator { + void translateNode(TransformT transform, FlinkBatchTranslationContext context); } private static String genSpaces(int n) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java index a03352efae15..86aee84ab1cb 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java @@ -17,81 +17,88 @@ */ 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.FlinkAssignWindows; 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.FlinkMergingNonShuffleReduceFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkMergingPartialReduceFunction; +import org.apache.beam.runners.flink.translation.functions.FlinkMergingReduceFunction; 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 org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGbkResultSchema; -import org.apache.beam.sdk.transforms.join.CoGroupByKey; -import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; import org.apache.beam.sdk.transforms.join.RawUnionValue; +import org.apache.beam.sdk.transforms.join.UnionCoder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; -import com.google.api.client.util.Maps; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; -import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -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.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.MapOperator; import org.apache.flink.api.java.operators.MapPartitionOperator; +import org.apache.flink.api.java.operators.SingleInputUdfOperator; import org.apache.flink.api.java.operators.UnsortedGrouping; import org.apache.flink.core.fs.Path; +import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; /** - * Translators for transforming - * Dataflow {@link org.apache.beam.sdk.transforms.PTransform}s to - * Flink {@link org.apache.flink.api.java.DataSet}s + * Translators for transforming {@link PTransform PTransforms} to + * Flink {@link DataSet DataSets}. */ public class FlinkBatchTransformTranslators { @@ -100,113 +107,90 @@ public class FlinkBatchTransformTranslators { // -------------------------------------------------------------------------------------------- @SuppressWarnings("rawtypes") - private static final Map, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>(); + private static final Map< + Class, + FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>(); - // 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(Create.Values.class, new CreateTranslatorBatch()); + TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch()); - // TODO we're currently ignoring windows here but that has to change in the future - TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch()); + TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslatorBatch()); - 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(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch()); 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) { + public static FlinkBatchPipelineTranslator.BatchTransformTranslator getTranslator( + PTransform transform) { return TRANSLATORS.get(transform.getClass()); } - private static class ReadSourceTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + 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); + TypeInformation> typeInformation = context.getTypeInfo(output); - DataSource dataSource = new DataSource<>(context.getExecutionEnvironment(), - new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name); + 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); + private static class WriteSinkTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { @Override - public void translateNode(AvroIO.Read.Bound transform, FlinkBatchTranslationContext context) { - String path = transform.getFilepattern(); + public void translateNode(Write.Bound transform, FlinkBatchTranslationContext context) { 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); + PValue input = context.getInput(transform); + DataSet> inputDataSet = context.getInputDataSet(input); - context.setOutputDataSet(output, source); + inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())) + .name(name); } } - private static class AvroIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + 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)); + 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: {}.", + LOG.warn( + "Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix); - LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate); + 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; @@ -221,8 +205,17 @@ public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslation throw new RuntimeException("Could not access type from AvroIO.Bound", e); } - DataSink dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path - (filenamePrefix), extractedAvroType)); + MapOperator, T> valueStream = inputDataSet.map( + new MapFunction, T>() { + @Override + public T map(WindowedValue value) throws Exception { + return value.getValue(); + } + }).returns(new CoderTypeInformation<>(context.getInput(transform).getCoder())); + + + DataSink dataSink = valueStream.output( + new AvroOutputFormat<>(new Path(filenamePrefix), extractedAvroType)); if (numShards > 0) { dataSink.setParallelism(numShards); @@ -230,37 +223,16 @@ public void translateNode(AvroIO.Write.Bound transform, FlinkBatchTranslation } } - 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 class TextIOWriteTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class); @Override - public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslationContext context) { + public void translateNode( + TextIO.Write.Bound transform, + FlinkBatchTranslationContext context) { PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); + DataSet> inputDataSet = context.getInputDataSet(input); String filenamePrefix = transform.getFilenamePrefix(); String filenameSuffix = transform.getFilenameSuffix(); @@ -269,12 +241,25 @@ public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslation 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); + 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); + MapOperator, T> valueStream = inputDataSet.map( + new MapFunction, T>() { + @Override + public T map(WindowedValue value) throws Exception { + return value.getValue(); + } + }).returns(new CoderTypeInformation<>(transform.getCoder())); + + DataSink dataSink = valueStream.writeAsText(filenamePrefix); if (numShards > 0) { dataSink.setParallelism(numShards); @@ -282,148 +267,412 @@ public void translateNode(TextIO.Write.Bound transform, FlinkBatchTranslation } } - private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator { + private static class WindowBoundTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + @Override - public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) { + public void translateNode(Window.Bound transform, FlinkBatchTranslationContext context) { PValue input = context.getInput(transform); - DataSet inputDataSet = context.getInputDataSet(input); - inputDataSet.printOnTaskManager(transform.getName()); + + TypeInformation> resultTypeInfo = + context.getTypeInfo(context.getOutput(transform)); + + DataSet> inputDataSet = context.getInputDataSet(input); + + @SuppressWarnings("unchecked") + final WindowingStrategy windowingStrategy = + (WindowingStrategy) + context.getOutput(transform).getWindowingStrategy(); + + WindowFn windowFn = windowingStrategy.getWindowFn(); + + FlinkAssignWindows assignWindowsFunction = + new FlinkAssignWindows<>(windowFn); + + DataSet> resultDataSet = inputDataSet + .flatMap(assignWindowsFunction) + .name(context.getOutput(transform).getName()) + .returns(resultTypeInfo); + + context.setOutputDataSet(context.getOutput(transform), resultDataSet); } } - private static class WriteSinkTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class GroupByKeyTranslatorBatch + 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); + public void translateNode( + GroupByKey transform, + FlinkBatchTranslationContext context) { + + // for now, this is copied from the Combine.PerKey translater. Once we have the new runner API + // we can replace GroupByKey by a Combine.PerKey with the Concatenate CombineFn + + DataSet>> inputDataSet = + context.getInputDataSet(context.getInput(transform)); + + Combine.KeyedCombineFn, List> combineFn = + new Concatenate().asKeyedFn(); + + KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); + + Coder> accumulatorCoder; + + try { + accumulatorCoder = + combineFn.getAccumulatorCoder( + context.getInput(transform).getPipeline().getCoderRegistry(), + inputCoder.getKeyCoder(), + inputCoder.getValueCoder()); + } catch (CannotProvideCoderException e) { + throw new RuntimeException(e); + } + + WindowingStrategy windowingStrategy = + context.getInput(transform).getWindowingStrategy(); + + TypeInformation>> kvCoderTypeInformation = + new KvCoderTypeInformation<>( + WindowedValue.getFullCoder( + inputCoder, + windowingStrategy.getWindowFn().windowCoder())); + + TypeInformation>>> partialReduceTypeInfo = + new KvCoderTypeInformation<>( + WindowedValue.getFullCoder( + KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder), + windowingStrategy.getWindowFn().windowCoder())); + + Grouping>> inputGrouping = + new UnsortedGrouping<>( + inputDataSet, + new Keys.ExpressionKeys<>(new String[]{"key"}, + kvCoderTypeInformation)); + + FlinkPartialReduceFunction, ?> partialReduceFunction; + FlinkReduceFunction, List, ?> reduceFunction; + + if (windowingStrategy.getWindowFn().isNonMerging()) { + @SuppressWarnings("unchecked") + WindowingStrategy boundedStrategy = + (WindowingStrategy) windowingStrategy; + + partialReduceFunction = new FlinkPartialReduceFunction<>( + combineFn, + boundedStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + + reduceFunction = new FlinkReduceFunction<>( + combineFn, + boundedStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + + } else { + if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) { + throw new UnsupportedOperationException( + "Merging WindowFn with windows other than IntervalWindow are not supported."); + } + + @SuppressWarnings("unchecked") + WindowingStrategy intervalStrategy = + (WindowingStrategy) windowingStrategy; + + partialReduceFunction = new FlinkMergingPartialReduceFunction<>( + combineFn, + intervalStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + + reduceFunction = new FlinkMergingReduceFunction<>( + combineFn, + intervalStrategy, + Collections., WindowingStrategy>emptyMap(), + context.getPipelineOptions()); + } + + // Partially GroupReduce the values into the intermediate format AccumT (combine) + GroupCombineOperator< + WindowedValue>, + WindowedValue>>> groupCombine = + new GroupCombineOperator<>( + inputGrouping, + partialReduceTypeInfo, + partialReduceFunction, + "GroupCombine: " + transform.getName()); + + Grouping>>> intermediateGrouping = + new UnsortedGrouping<>( + groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); + + // Fully reduce the values and create output format VO + GroupReduceOperator< + WindowedValue>>, WindowedValue>>> outputDataSet = + new GroupReduceOperator<>( + intermediateGrouping, partialReduceTypeInfo, reduceFunction, transform.getName()); + + context.setOutputDataSet(context.getOutput(transform), outputDataSet); - inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name); } } /** - * Translates a GroupByKey while ignoring window assignments. Current ignores windows. + * Combiner that combines {@code T}s into a single {@code List} containing all inputs. + * + *

For internal use to translate {@link GroupByKey}. For a large {@link PCollection} this + * is expected to crash! + * + *

This is copied from the dataflow runner code. + * + * @param the type of elements to concatenate. */ - private static class GroupByKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class Concatenate extends Combine.CombineFn, List> { + @Override + public List createAccumulator() { + return new ArrayList(); + } @Override - public void translateNode(GroupByKey transform, FlinkBatchTranslationContext context) { - DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); - GroupReduceFunction, KV>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>(); + public List addInput(List accumulator, T input) { + accumulator.add(input); + return accumulator; + } - TypeInformation>> typeInformation = context.getTypeInfo(context.getOutput(transform)); + @Override + public List mergeAccumulators(Iterable> accumulators) { + List result = createAccumulator(); + for (List accumulator : accumulators) { + result.addAll(accumulator); + } + return result; + } - Grouping> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType())); + @Override + public List extractOutput(List accumulator) { + return accumulator; + } - GroupReduceOperator, KV>> outputDataSet = - new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName()); + @Override + public Coder> getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { + return ListCoder.of(inputCoder); + } - context.setOutputDataSet(context.getOutput(transform), outputDataSet); + @Override + public Coder> getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) { + return ListCoder.of(inputCoder); } } - private static class CombinePerKeyTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + + private static class CombinePerKeyTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + Combine.PerKey> { @Override - public void translateNode(Combine.PerKey transform, FlinkBatchTranslationContext context) { - DataSet> inputDataSet = context.getInputDataSet(context.getInput(transform)); + @SuppressWarnings("unchecked") + public void translateNode( + Combine.PerKey transform, + FlinkBatchTranslationContext context) { + DataSet>> inputDataSet = + context.getInputDataSet(context.getInput(transform)); - @SuppressWarnings("unchecked") - Combine.KeyedCombineFn keyedCombineFn = (Combine.KeyedCombineFn) transform.getFn(); + CombineFnBase.PerKeyCombineFn combineFn = + (CombineFnBase.PerKeyCombineFn) transform.getFn(); - KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); + KvCoder inputCoder = (KvCoder) context.getInput(transform).getCoder(); + + Coder accumulatorCoder; - Coder accumulatorCoder = - null; try { - accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder()); + accumulatorCoder = + combineFn.getAccumulatorCoder( + context.getInput(transform).getPipeline().getCoderRegistry(), + inputCoder.getKeyCoder(), + inputCoder.getValueCoder()); } catch (CannotProvideCoderException e) { - e.printStackTrace(); - // TODO + throw new RuntimeException(e); } - TypeInformation> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder); - TypeInformation> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder)); + WindowingStrategy windowingStrategy = + context.getInput(transform).getWindowingStrategy(); + + TypeInformation>> kvCoderTypeInformation = + new KvCoderTypeInformation<>( + WindowedValue.getFullCoder( + inputCoder, + windowingStrategy.getWindowFn().windowCoder())); + + TypeInformation>> partialReduceTypeInfo = + new KvCoderTypeInformation<>( + WindowedValue.getFullCoder( + KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder), + windowingStrategy.getWindowFn().windowCoder())); + + Grouping>> inputGrouping = + new UnsortedGrouping<>( + inputDataSet, + new Keys.ExpressionKeys<>(new String[]{"key"}, + kvCoderTypeInformation)); + + // construct a map from side input to WindowingStrategy so that + // the DoFn runner can map main-input windows to side input windows + Map, WindowingStrategy> sideInputStrategies = new HashMap<>(); + for (PCollectionView sideInput: transform.getSideInputs()) { + sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); + } - Grouping> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation)); + if (windowingStrategy.getWindowFn().isNonMerging()) { + WindowingStrategy boundedStrategy = + (WindowingStrategy) windowingStrategy; + + FlinkPartialReduceFunction partialReduceFunction = + new FlinkPartialReduceFunction<>( + combineFn, + boundedStrategy, + sideInputStrategies, + context.getPipelineOptions()); + + FlinkReduceFunction reduceFunction = + new FlinkReduceFunction<>( + combineFn, + boundedStrategy, + sideInputStrategies, + context.getPipelineOptions()); + + // Partially GroupReduce the values into the intermediate format AccumT (combine) + GroupCombineOperator< + WindowedValue>, + WindowedValue>> groupCombine = + new GroupCombineOperator<>( + inputGrouping, + partialReduceTypeInfo, + partialReduceFunction, + "GroupCombine: " + transform.getName()); + + transformSideInputs(transform.getSideInputs(), groupCombine, context); + + TypeInformation>> reduceTypeInfo = + context.getTypeInfo(context.getOutput(transform)); + + Grouping>> intermediateGrouping = + new UnsortedGrouping<>( + groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType())); + + // Fully reduce the values and create output format OutputT + GroupReduceOperator< + WindowedValue>, WindowedValue>> outputDataSet = + new GroupReduceOperator<>( + intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName()); + + transformSideInputs(transform.getSideInputs(), outputDataSet, context); + + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + + } else { + if (!windowingStrategy.getWindowFn().windowCoder().equals(IntervalWindow.getCoder())) { + throw new UnsupportedOperationException( + "Merging WindowFn with windows other than IntervalWindow are not supported."); + } - FlinkPartialReduceFunction partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn); + // for merging windows we can't to a pre-shuffle combine step since + // elements would not be in their correct windows for side-input access - // Partially GroupReduce the values into the intermediate format VA (combine) - GroupCombineOperator, KV> groupCombine = - new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction, - "GroupCombine: " + transform.getName()); + WindowingStrategy intervalStrategy = + (WindowingStrategy) windowingStrategy; - // Reduce fully to VO - GroupReduceFunction, KV> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn); + FlinkMergingNonShuffleReduceFunction reduceFunction = + new FlinkMergingNonShuffleReduceFunction<>( + combineFn, + intervalStrategy, + sideInputStrategies, + context.getPipelineOptions()); - 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>> grouping = + new UnsortedGrouping<>( + inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation)); + + // Fully reduce the values and create output format OutputT + GroupReduceOperator< + WindowedValue>, WindowedValue>> outputDataSet = + new GroupReduceOperator<>( + grouping, reduceTypeInfo, reduceFunction, transform.getName()); + + transformSideInputs(transform.getSideInputs(), outputDataSet, context); + + context.setOutputDataSet(context.getOutput(transform), outputDataSet); + } - // 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); } } -// private static class CombineGroupedValuesTranslator implements FlinkPipelineTranslator.TransformTranslator> { -// -// @Override -// public void translateNode(Combine.GroupedValues transform, TranslationContext context) { -// DataSet> inputDataSet = context.getInputDataSet(transform.getInput()); -// -// Combine.KeyedCombineFn keyedCombineFn = transform.getFn(); -// -// GroupReduceFunction, KV> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn); -// -// TypeInformation> typeInformation = context.getTypeInfo(transform.getOutput()); -// -// 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); + private static class ParDoBoundTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + ParDo.Bound> { @Override - public void translateNode(ParDo.Bound transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); + public void translateNode( + ParDo.Bound transform, + FlinkBatchTranslationContext context) { + DataSet> inputDataSet = + context.getInputDataSet(context.getInput(transform)); - final DoFn doFn = transform.getFn(); + final DoFn doFn = transform.getFn(); - TypeInformation typeInformation = context.getTypeInfo(context.getOutput(transform)); + TypeInformation> typeInformation = + context.getTypeInfo(context.getOutput(transform)); - FlinkDoFnFunction doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions()); - MapPartitionOperator outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName()); + List> sideInputs = transform.getSideInputs(); - transformSideInputs(transform.getSideInputs(), outputDataSet, context); + // construct a map from side input to WindowingStrategy so that + // the DoFn runner can map main-input windows to side input windows + Map, WindowingStrategy> sideInputStrategies = new HashMap<>(); + for (PCollectionView sideInput: sideInputs) { + sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); + } + + FlinkDoFnFunction doFnWrapper = + new FlinkDoFnFunction<>( + doFn, + context.getOutput(transform).getWindowingStrategy(), + sideInputStrategies, + context.getPipelineOptions()); + + MapPartitionOperator, WindowedValue> outputDataSet = + new MapPartitionOperator<>( + inputDataSet, + typeInformation, + doFnWrapper, + transform.getName()); + + transformSideInputs(sideInputs, outputDataSet, context); context.setOutputDataSet(context.getOutput(transform), outputDataSet); } } - private static class ParDoBoundMultiTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { - private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslatorBatch.class); + private static class ParDoBoundMultiTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + ParDo.BoundMulti> { @Override - public void translateNode(ParDo.BoundMulti transform, FlinkBatchTranslationContext context) { - DataSet inputDataSet = context.getInputDataSet(context.getInput(transform)); + public void translateNode( + ParDo.BoundMulti transform, + FlinkBatchTranslationContext context) { + DataSet> inputDataSet = + context.getInputDataSet(context.getInput(transform)); - final DoFn doFn = transform.getFn(); + 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 + // put the main output at index 0, FlinkMultiOutputDoFnFunction expects this outputMap.put(transform.getMainOutputTag(), 0); int count = 1; for (TupleTag tag: outputs.keySet()) { @@ -432,147 +681,166 @@ public void translateNode(ParDo.BoundMulti transform, FlinkBatchTransla } } + // assume that the windowing strategy is the same for all outputs + WindowingStrategy windowingStrategy = null; + // 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()); + windowingStrategy = coll.getWindowingStrategy(); + } + + if (windowingStrategy == null) { + throw new IllegalStateException("No outputs defined."); } UnionCoder unionCoder = UnionCoder.of(outputCoders); - @SuppressWarnings("unchecked") - TypeInformation typeInformation = new CoderTypeInformation<>(unionCoder); + TypeInformation> typeInformation = + new CoderTypeInformation<>( + WindowedValue.getFullCoder( + unionCoder, + windowingStrategy.getWindowFn().windowCoder())); - @SuppressWarnings("unchecked") - FlinkMultiOutputDoFnFunction doFnWrapper = new FlinkMultiOutputDoFnFunction(doFn, context.getPipelineOptions(), outputMap); - MapPartitionOperator outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName()); + List> sideInputs = transform.getSideInputs(); - transformSideInputs(transform.getSideInputs(), outputDataSet, context); + // construct a map from side input to WindowingStrategy so that + // the DoFn runner can map main-input windows to side input windows + Map, WindowingStrategy> sideInputStrategies = new HashMap<>(); + for (PCollectionView sideInput: sideInputs) { + sideInputStrategies.put(sideInput, sideInput.getWindowingStrategyInternal()); + } - 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); + @SuppressWarnings("unchecked") + FlinkMultiOutputDoFnFunction doFnWrapper = + new FlinkMultiOutputDoFnFunction( + doFn, + windowingStrategy, + sideInputStrategies, + context.getPipelineOptions(), + outputMap); + + MapPartitionOperator, WindowedValue> taggedDataSet = + new MapPartitionOperator<>( + inputDataSet, + typeInformation, + doFnWrapper, + transform.getName()); + + transformSideInputs(sideInputs, taggedDataSet, context); + for (Map.Entry, PCollection> output: outputs.entrySet()) { + pruneOutput( + taggedDataSet, + context, + outputMap.get(output.getKey()), + (PCollection) output.getValue()); } } + + private void pruneOutput( + MapPartitionOperator, WindowedValue> taggedDataSet, + FlinkBatchTranslationContext context, + int integerTag, + PCollection collection) { + TypeInformation> outputType = context.getTypeInfo(collection); + + FlinkMultiOutputPruningFunction pruningFunction = + new FlinkMultiOutputPruningFunction<>(integerTag); + + FlatMapOperator, WindowedValue> pruningOperator = + new FlatMapOperator<>( + taggedDataSet, + outputType, + pruningFunction, + collection.getName()); + + context.setOutputDataSet(collection, pruningOperator); + } } - private static class FlattenPCollectionTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class FlattenPCollectionTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + Flatten.FlattenPCollectionList> { @Override - public void translateNode(Flatten.FlattenPCollectionList transform, FlinkBatchTranslationContext context) { + @SuppressWarnings("unchecked") + 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); + DataSet> result = null; + + if (allInputs.isEmpty()) { + + // create an empty dummy source to satisfy downstream operations + // we cannot create an empty source in Flink, therefore we have to + // add the flatMap that simply never forwards the single element + DataSource dummySource = + context.getExecutionEnvironment().fromElements("dummy"); + result = dummySource.flatMap(new FlatMapFunction>() { + @Override + public void flatMap(String s, Collector> collector) throws Exception { + // never return anything + } + }).returns( + new CoderTypeInformation<>( + WindowedValue.getFullCoder( + (Coder) VoidCoder.of(), + GlobalWindow.Coder.INSTANCE))); + } else { + 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); + // insert a dummy filter, there seems to be a bug in Flink + // that produces duplicate elements after the union in some cases + // if we don't + result = result.filter(new FilterFunction>() { + @Override + public boolean filter(WindowedValue tWindowedValue) throws Exception { + return true; + } + }).name("UnionFixFilter"); + context.setOutputDataSet(context.getOutput(transform), result); } } - private static class CreateTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator> { + private static class CreatePCollectionViewTranslatorBatch + implements FlinkBatchPipelineTranslator.BatchTransformTranslator< + View.CreatePCollectionView> { @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); - } - } + public void translateNode( + View.CreatePCollectionView transform, + FlinkBatchTranslationContext context) { + DataSet> inputDataSet = + context.getInputDataSet(context.getInput(transform)); - DataSet initDataSet = context.getExecutionEnvironment().fromElements(1); - FlinkCreateFunction flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder); - FlatMapOperator outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName()); + PCollectionView input = transform.getView(); - context.setOutputDataSet(context.getOutput(transform), outputDataSet); + context.setSideInputDataSet(input, inputDataSet); } } - private static void transformSideInputs(List> sideInputs, - MapPartitionOperator outputDataSet, - FlinkBatchTranslationContext context) { + private static void transformSideInputs( + List> sideInputs, + SingleInputUdfOperator outputDataSet, + FlinkBatchTranslationContext 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()); } } -// 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> { - - @Override - public void translateNode(CoGroupByKey transform, FlinkBatchTranslationContext context) { - KeyedPCollectionTuple input = context.getInput(transform); - - 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.getOutputTypeInfo(); - - 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(context.getOutput(transform), out); - } - } - - // -------------------------------------------------------------------------------------------- - // Miscellaneous - // -------------------------------------------------------------------------------------------- - private FlinkBatchTransformTranslators() {} } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java index 71950cf216cb..ecc3a65c7965 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java @@ -18,31 +18,40 @@ package org.apache.beam.runners.flink.translation; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; -import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; -import org.apache.beam.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; +/** + * Helper for {@link FlinkBatchPipelineTranslator} and translators in + * {@link FlinkBatchTransformTranslators}. + */ public class FlinkBatchTranslationContext { private final Map> dataSets; private final Map, DataSet> broadcastDataSets; + /** + * For keeping track about which DataSets don't have a successor. We + * need to terminate these with a discarding sink because the Beam + * model allows dangling operations. + */ + private final Map> danglingDataSets; + private final ExecutionEnvironment env; private final PipelineOptions options; @@ -55,10 +64,16 @@ public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions op this.options = options; this.dataSets = new HashMap<>(); this.broadcastDataSets = new HashMap<>(); + + this.danglingDataSets = new HashMap<>(); } // ------------------------------------------------------------------------ - + + public Map> getDanglingDataSets() { + return danglingDataSets; + } + public ExecutionEnvironment getExecutionEnvironment() { return env; } @@ -68,13 +83,16 @@ public PipelineOptions getPipelineOptions() { } @SuppressWarnings("unchecked") - public DataSet getInputDataSet(PValue value) { - return (DataSet) dataSets.get(value); + public DataSet> getInputDataSet(PValue value) { + // assume that the DataSet is used as an input if retrieved here + danglingDataSets.remove(value); + return (DataSet>) dataSets.get(value); } - public void setOutputDataSet(PValue value, DataSet set) { + public void setOutputDataSet(PValue value, DataSet> set) { if (!dataSets.containsKey(value)) { dataSets.put(value, set); + danglingDataSets.put(value, set); } } @@ -91,40 +109,32 @@ 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); } } - - @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()); - } + @SuppressWarnings("unchecked") + public TypeInformation> getTypeInfo(PCollection collection) { + Coder valueCoder = collection.getCoder(); + WindowedValue.FullWindowedValueCoder windowedValueCoder = + WindowedValue.getFullCoder( + valueCoder, + collection.getWindowingStrategy().getWindowFn().windowCoder()); - public TypeInformation getOutputTypeInfo() { - return getTypeInfo((PValue) currentTransform.getOutput()); + return new CoderTypeInformation<>(windowedValueCoder); } @SuppressWarnings("unchecked") - I getInput(PTransform transform) { - return (I) currentTransform.getInput(); + T getInput(PTransform transform) { + return (T) currentTransform.getInput(); } @SuppressWarnings("unchecked") - O getOutput(PTransform transform) { - return (O) currentTransform.getOutput(); + T getOutput(PTransform transform) { + return (T) currentTransform.getOutput(); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java index 618727dae4d9..7aedf5520502 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java @@ -18,7 +18,6 @@ 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.types.FlinkCoder; import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat; @@ -45,6 +44,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.join.RawUnionValue; +import org.apache.beam.sdk.transforms.join.UnionCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -228,29 +228,15 @@ public void translateNode(Read.Bounded transform, FlinkStreamingTranslationCo BoundedSource boundedSource = transform.getSource(); PCollection output = context.getOutput(transform); - Coder defaultOutputCoder = boundedSource.getDefaultOutputCoder(); - CoderTypeInformation typeInfo = new CoderTypeInformation<>(defaultOutputCoder); + TypeInformation> typeInfo = context.getTypeInfo(output); - DataStream source = context.getExecutionEnvironment().createInput( + DataStream> source = context.getExecutionEnvironment().createInput( new SourceInputFormat<>( boundedSource, context.getPipelineOptions()), typeInfo); - DataStream> windowedStream = source.flatMap( - new FlatMapFunction>() { - @Override - public void flatMap(T value, Collector> out) throws Exception { - out.collect( - WindowedValue.of(value, - Instant.now(), - GlobalWindow.INSTANCE, - PaneInfo.NO_FIRING)); - } - }) - .assignTimestampsAndWatermarks(new IngestionTimeExtractor>()); - - context.setOutputDataStream(output, windowedStream); + context.setOutputDataStream(output, source); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java index 8bc73172405c..0cb80baa7cc8 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java @@ -17,21 +17,30 @@ */ package org.apache.beam.runners.flink.translation; +import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import com.google.common.base.Preconditions; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import java.util.HashMap; import java.util.Map; +/** + * Helper for keeping track of which {@link DataStream DataStreams} map + * to which {@link PTransform PTransforms}. + */ public class FlinkStreamingTranslationContext { private final StreamExecutionEnvironment env; @@ -80,12 +89,24 @@ public void setCurrentTransform(AppliedPTransform currentTransform) { } @SuppressWarnings("unchecked") - public I getInput(PTransform transform) { - return (I) currentTransform.getInput(); + public TypeInformation> getTypeInfo(PCollection collection) { + Coder valueCoder = collection.getCoder(); + WindowedValue.FullWindowedValueCoder windowedValueCoder = + WindowedValue.getFullCoder( + valueCoder, + collection.getWindowingStrategy().getWindowFn().windowCoder()); + + return new CoderTypeInformation<>(windowedValueCoder); + } + + + @SuppressWarnings("unchecked") + public T getInput(PTransform transform) { + return (T) currentTransform.getInput(); } @SuppressWarnings("unchecked") - public O getOutput(PTransform transform) { - return (O) currentTransform.getOutput(); + public T getOutput(PTransform transform) { + return (T) currentTransform.getOutput(); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java new file mode 100644 index 000000000000..7ea8c202f9d5 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.translation.functions; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.WindowedValue; + +import org.joda.time.Instant; + +import java.util.Collection; + +/** + * {@link org.apache.beam.sdk.transforms.windowing.WindowFn.AssignContext} for + * Flink functions. + */ +class FlinkAssignContext + extends WindowFn.AssignContext { + private final WindowedValue value; + + FlinkAssignContext(WindowFn fn, WindowedValue value) { + fn.super(); + this.value = value; + } + + @Override + public InputT element() { + return value.getValue(); + } + + @Override + public Instant timestamp() { + return value.getTimestamp(); + } + + @Override + public Collection windows() { + return value.getWindows(); + } + +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java new file mode 100644 index 000000000000..e07e49a2f060 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.translation.functions; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.WindowedValue; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.util.Collector; + +import java.util.Collection; + +/** + * Flink {@link FlatMapFunction} for implementing + * {@link org.apache.beam.sdk.transforms.windowing.Window.Bound}. + */ +public class FlinkAssignWindows + implements FlatMapFunction, WindowedValue> { + + private final WindowFn windowFn; + + public FlinkAssignWindows(WindowFn windowFn) { + this.windowFn = windowFn; + } + + @Override + public void flatMap( + WindowedValue input, Collector> collector) throws Exception { + Collection windows = windowFn.assignWindows(new FlinkAssignContext<>(windowFn, input)); + for (W window: windows) { + collector.collect( + WindowedValue.of(input.getValue(), input.getTimestamp(), window, input.getPane())); + } + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java deleted file mode 100644 index 8e7cdd75ca48..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.functions; - -import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGbkResultSchema; -import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.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; - - 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))); - } -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java deleted file mode 100644 index e5ac7482cfcb..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.functions; - -import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; -import org.apache.beam.sdk.coders.Coder; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.util.Collector; - -import java.io.ByteArrayInputStream; -import java.util.List; - -/** - * This is a hack for transforming a {@link org.apache.beam.sdk.transforms.Create} - * operation. Flink does not allow {@code null} in it's equivalent operation: - * {@link org.apache.flink.api.java.ExecutionEnvironment#fromElements(Object[])}. Therefore - * we use a DataSource with one dummy element and output the elements of the Create operation - * inside this FlatMap. - */ -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(); - } -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index 3566f7e1070e..89243a3ede28 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -18,173 +18,85 @@ package org.apache.beam.runners.flink.translation.functions; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; -import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingInternals; -import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.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.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; +import java.util.Map; /** * Encapsulates a {@link org.apache.beam.sdk.transforms.DoFn} * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}. */ -public class FlinkDoFnFunction extends RichMapPartitionFunction { +public class FlinkDoFnFunction + extends RichMapPartitionFunction, WindowedValue> { - private final DoFn doFn; + private final DoFn doFn; private final SerializedPipelineOptions serializedOptions; - public FlinkDoFnFunction(DoFn doFn, PipelineOptions options) { - this.doFn = doFn; - this.serializedOptions = new SerializedPipelineOptions(options); - } - - @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; - } - + private final Map, WindowingStrategy> sideInputs; - @Override - public Instant timestamp() { - return Instant.now(); - } + private final boolean requiresWindowAccess; + private final boolean hasSideInputs; - @Override - public BoundedWindow window() { - return GlobalWindow.INSTANCE; - } - - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } + private final WindowingStrategy windowingStrategy; - @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; - } + public FlinkDoFnFunction( + DoFn doFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions options) { + this.doFn = doFn; + this.sideInputs = sideInputs; + this.serializedOptions = new SerializedPipelineOptions(options); + this.windowingStrategy = windowingStrategy; - @Override - public Collection windows() { - return ImmutableList.of(GlobalWindow.INSTANCE); - } + this.requiresWindowAccess = doFn instanceof DoFn.RequiresWindowAccess; + this.hasSideInputs = !sideInputs.isEmpty(); + } - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } + @Override + public void mapPartition( + Iterable> values, + Collector> out) throws Exception { + + FlinkProcessContext context = new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); - @Override - public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { - } + this.doFn.startBundle(context); - @Override - public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { - throw new RuntimeException("sideInput() not implemented."); + if (!requiresWindowAccess || hasSideInputs) { + // we don't need to explode the windows + for (WindowedValue value : values) { + context = context.forWindowedValue(value); + doFn.processElement(context); + } + } else { + // we need to explode the windows because we have per-window + // side inputs and window access also only works if an element + // is in only one window + for (WindowedValue value : values) { + for (WindowedValue explodedValue: value.explodeWindows()) { + context = context.forWindowedValue(value); + doFn.processElement(context); } - }; - } - - @Override - public PipelineOptions getPipelineOptions() { - return serializedOptions.getPipelineOptions(); - } - - @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; - } - - + // set the windowed value to null so that the logic + // or outputting in finishBundle kicks in + context = context.forWindowedValue(null); + this.doFn.finishBundle(context); } + } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java deleted file mode 100644 index 7c7084db287c..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.functions; - -import org.apache.beam.sdk.values.KV; - -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 - * {@link org.apache.beam.sdk.transforms.GroupByKey} operation. This reads the input - * {@link org.apache.beam.sdk.values.KV} elements, extracts the key and collects - * the values in a {@code List}. - */ -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)); - } - - 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."); - } - } -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java new file mode 100644 index 000000000000..9074d72e0e15 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.translation.functions; + +import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.CombineFnBase; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import org.apache.flink.api.common.functions.RichGroupReduceFunction; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * Special version of {@link FlinkReduceFunction} that supports merging windows. This + * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the + * same behaviour as {@code MergeOverlappingIntervalWindows}. + * + *

This is different from the pair of function for the non-merging windows case + * in that we cannot do combining before the shuffle because elements would not + * yet be in their correct windows for side-input access. + */ +public class FlinkMergingNonShuffleReduceFunction< + K, InputT, AccumT, OutputT, W extends IntervalWindow> + extends RichGroupReduceFunction>, WindowedValue>> { + + private final CombineFnBase.PerKeyCombineFn combineFn; + + private final DoFn, KV> doFn; + + private final WindowingStrategy windowingStrategy; + + private final Map, WindowingStrategy> sideInputs; + + private final SerializedPipelineOptions serializedOptions; + + public FlinkMergingNonShuffleReduceFunction( + CombineFnBase.PerKeyCombineFn keyedCombineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { + + this.combineFn = keyedCombineFn; + + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + + this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + + // dummy DoFn because we need one for ProcessContext + this.doFn = new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) throws Exception { + + } + }; + } + + @Override + public void reduce( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); + + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) windowingStrategy.getOutputTimeFn(); + + // get all elements so that we can sort them, has to fit into + // memory + // this seems very unprudent, but correct, for now + List>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + sortedInput.add(exploded); + } + } + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> o2) { + return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp() + .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp()); + } + }); + + // merge windows, we have to do it in an extra pre-processing step and + // can't do it as we go since the window of early elements would not + // be correct when calling the CombineFn + mergeWindow(sortedInput); + + // iterate over the elements that are sorted by window timestamp + final Iterator>> iterator = sortedInput.iterator(); + + // create accumulator using the first elements key + WindowedValue> currentValue = iterator.next(); + K key = currentValue.getValue().getKey(); + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows()); + InputT firstValue = currentValue.getValue().getValue(); + processContext = processContext.forWindowedValue(currentValue); + AccumT accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, firstValue, processContext); + + // we use this to keep track of the timestamps assigned by the OutputTimeFn + Instant windowTimestamp = + outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + IntervalWindow nextWindow = (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + + if (currentWindow.equals(nextWindow)) { + // continue accumulating and merge windows + + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + + windowTimestamp = outputTimeFn.combine( + windowTimestamp, + outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow)); + + } else { + // emit the value that we currently have + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + windowTimestamp, + currentWindow, + PaneInfo.NO_FIRING)); + + currentWindow = nextWindow; + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); + } + } + + // emit the final accumulator + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + windowTimestamp, + currentWindow, + PaneInfo.NO_FIRING)); + } + + /** + * Merge windows. This assumes that the list of elements is sorted by window-end timestamp. + * This replaces windows in the input list. + */ + private void mergeWindow(List>> elements) { + int currentStart = 0; + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows()); + + for (int i = 1; i < elements.size(); i++) { + WindowedValue> nextValue = elements.get(i); + IntervalWindow nextWindow = + (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + if (currentWindow.intersects(nextWindow)) { + // we continue + currentWindow = currentWindow.span(nextWindow); + } else { + // retrofit the merged window to all windows up to "currentStart" + for (int j = i - 1; j >= currentStart; j--) { + WindowedValue> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + currentStart = i; + currentWindow = nextWindow; + } + } + if (currentStart < elements.size() - 1) { + // we have to retrofit the last batch + for (int j = elements.size() - 1; j >= currentStart; j--) { + WindowedValue> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + } + } + +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java new file mode 100644 index 000000000000..c12e4204a3f0 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingPartialReduceFunction.java @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.translation.functions; + +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.CombineFnBase; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * Special version of {@link FlinkPartialReduceFunction} that supports merging windows. This + * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the + * same behaviour as {@code MergeOverlappingIntervalWindows}. + */ +public class FlinkMergingPartialReduceFunction + extends FlinkPartialReduceFunction { + + public FlinkMergingPartialReduceFunction( + CombineFnBase.PerKeyCombineFn combineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { + super(combineFn, windowingStrategy, sideInputs, pipelineOptions); + } + + @Override + public void combine( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); + + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) windowingStrategy.getOutputTimeFn(); + + // get all elements so that we can sort them, has to fit into + // memory + // this seems very unprudent, but correct, for now + List>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + sortedInput.add(exploded); + } + } + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> o2) { + return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp() + .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp()); + } + }); + + // merge windows, we have to do it in an extra pre-processing step and + // can't do it as we go since the window of early elements would not + // be correct when calling the CombineFn + mergeWindow(sortedInput); + + // iterate over the elements that are sorted by window timestamp + final Iterator>> iterator = sortedInput.iterator(); + + // create accumulator using the first elements key + WindowedValue> currentValue = iterator.next(); + K key = currentValue.getValue().getKey(); + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows()); + InputT firstValue = currentValue.getValue().getValue(); + processContext = processContext.forWindowedValue(currentValue); + AccumT accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, firstValue, processContext); + + // we use this to keep track of the timestamps assigned by the OutputTimeFn + Instant windowTimestamp = + outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + IntervalWindow nextWindow = (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + + if (currentWindow.equals(nextWindow)) { + // continue accumulating and merge windows + + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + + windowTimestamp = outputTimeFn.combine( + windowTimestamp, + outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow)); + + } else { + // emit the value that we currently have + out.collect( + WindowedValue.of( + KV.of(key, accumulator), + windowTimestamp, + currentWindow, + PaneInfo.NO_FIRING)); + + currentWindow = nextWindow; + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); + } + } + + // emit the final accumulator + out.collect( + WindowedValue.of( + KV.of(key, accumulator), + windowTimestamp, + currentWindow, + PaneInfo.NO_FIRING)); + } + + /** + * Merge windows. This assumes that the list of elements is sorted by window-end timestamp. + * This replaces windows in the input list. + */ + private void mergeWindow(List>> elements) { + int currentStart = 0; + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows()); + + for (int i = 1; i < elements.size(); i++) { + WindowedValue> nextValue = elements.get(i); + IntervalWindow nextWindow = + (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + if (currentWindow.intersects(nextWindow)) { + // we continue + currentWindow = currentWindow.span(nextWindow); + } else { + // retrofit the merged window to all windows up to "currentStart" + for (int j = i - 1; j >= currentStart; j--) { + WindowedValue> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + currentStart = i; + currentWindow = nextWindow; + } + } + if (currentStart < elements.size() - 1) { + // we have to retrofit the last batch + for (int j = elements.size() - 1; j >= currentStart; j--) { + WindowedValue> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + } + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java new file mode 100644 index 000000000000..07d1c9741533 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingReduceFunction.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.translation.functions; + +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.CombineFnBase; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * Special version of {@link FlinkReduceFunction} that supports merging windows. This + * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the + * same behaviour as {@code MergeOverlappingIntervalWindows}. + */ +public class FlinkMergingReduceFunction + extends FlinkReduceFunction { + + public FlinkMergingReduceFunction( + CombineFnBase.PerKeyCombineFn keyedCombineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { + super(keyedCombineFn, windowingStrategy, sideInputs, pipelineOptions); + } + + @Override + public void reduce( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); + + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) windowingStrategy.getOutputTimeFn(); + + + // get all elements so that we can sort them, has to fit into + // memory + // this seems very unprudent, but correct, for now + ArrayList>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + sortedInput.add(exploded); + } + } + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> o2) { + return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp() + .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp()); + } + }); + + // merge windows, we have to do it in an extra pre-processing step and + // can't do it as we go since the window of early elements would not + // be correct when calling the CombineFn + mergeWindow(sortedInput); + + // iterate over the elements that are sorted by window timestamp + final Iterator>> iterator = sortedInput.iterator(); + + // get the first accumulator + WindowedValue> currentValue = iterator.next(); + K key = currentValue.getValue().getKey(); + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(currentValue.getWindows()); + AccumT accumulator = currentValue.getValue().getValue(); + + // we use this to keep track of the timestamps assigned by the OutputTimeFn, + // in FlinkPartialReduceFunction we already merge the timestamps assigned + // to individual elements, here we just merge them + List windowTimestamps = new ArrayList<>(); + windowTimestamps.add(currentValue.getTimestamp()); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + IntervalWindow nextWindow = + (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + + if (nextWindow.equals(currentWindow)) { + // continue accumulating and merge windows + + processContext = processContext.forWindowedValue(nextValue); + + accumulator = combineFnRunner.mergeAccumulators( + key, ImmutableList.of(accumulator, nextValue.getValue().getValue()), processContext); + + windowTimestamps.add(nextValue.getTimestamp()); + } else { + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + outputTimeFn.merge(currentWindow, windowTimestamps), + currentWindow, + PaneInfo.NO_FIRING)); + + windowTimestamps.clear(); + + processContext = processContext.forWindowedValue(nextValue); + + currentWindow = nextWindow; + accumulator = nextValue.getValue().getValue(); + windowTimestamps.add(nextValue.getTimestamp()); + } + } + + // emit the final accumulator + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + outputTimeFn.merge(currentWindow, windowTimestamps), + currentWindow, + PaneInfo.NO_FIRING)); + } + + /** + * Merge windows. This assumes that the list of elements is sorted by window-end timestamp. + * This replaces windows in the input list. + */ + private void mergeWindow(List>> elements) { + int currentStart = 0; + IntervalWindow currentWindow = + (IntervalWindow) Iterables.getOnlyElement(elements.get(0).getWindows()); + + for (int i = 1; i < elements.size(); i++) { + WindowedValue> nextValue = elements.get(i); + IntervalWindow nextWindow = + (IntervalWindow) Iterables.getOnlyElement(nextValue.getWindows()); + if (currentWindow.intersects(nextWindow)) { + // we continue + currentWindow = currentWindow.span(nextWindow); + } else { + // retrofit the merged window to all windows up to "currentStart" + for (int j = i - 1; j >= currentStart; j--) { + WindowedValue> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + currentStart = i; + currentWindow = nextWindow; + } + } + if (currentStart < elements.size() - 1) { + // we have to retrofit the last batch + for (int j = elements.size() - 1; j >= currentStart; j--) { + WindowedValue> value = elements.get(j); + elements.set( + j, + WindowedValue.of( + value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + } + } + } + +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java index 476dc5e5f8e5..f92e76fa60cb 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java @@ -18,28 +18,17 @@ package org.apache.beam.runners.flink.translation.functions; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; -import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.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.ArrayList; -import java.util.List; import java.util.Map; /** @@ -50,112 +39,72 @@ * and must tag all outputs with the output number. Afterwards a filter will filter out * those elements that are not to be in a specific output. */ -public class FlinkMultiOutputDoFnFunction extends RichMapPartitionFunction { - - private final DoFn doFn; - private final SerializedPipelineOptions serializedPipelineOptions; - private final Map, Integer> outputMap; - - public FlinkMultiOutputDoFnFunction(DoFn doFn, PipelineOptions options, Map, Integer> outputMap) { - this.doFn = doFn; - this.serializedPipelineOptions = new SerializedPipelineOptions(options); - this.outputMap = outputMap; - } - - @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); - } +public class FlinkMultiOutputDoFnFunction + extends RichMapPartitionFunction, WindowedValue> { - private class ProcessContext extends DoFn.ProcessContext { + private final DoFn doFn; + private final SerializedPipelineOptions serializedOptions; - IN inValue; - Collector outCollector; + private final Map, Integer> outputMap; - public ProcessContext(DoFn fn, Collector outCollector) { - fn.super(); - this.outCollector = outCollector; - } + private final Map, WindowingStrategy> sideInputs; - @Override - public IN element() { - return this.inValue; - } + private final boolean requiresWindowAccess; + private final boolean hasSideInputs; - @Override - public Instant timestamp() { - return Instant.now(); - } + private final WindowingStrategy windowingStrategy; - @Override - public BoundedWindow window() { - return GlobalWindow.INSTANCE; - } + public FlinkMultiOutputDoFnFunction( + DoFn doFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions options, + Map, Integer> outputMap) { + this.doFn = doFn; + this.serializedOptions = new SerializedPipelineOptions(options); + this.outputMap = outputMap; - @Override - public PaneInfo pane() { - return PaneInfo.NO_FIRING; - } + this.requiresWindowAccess = doFn instanceof DoFn.RequiresWindowAccess; + this.hasSideInputs = !sideInputs.isEmpty(); + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + } - @Override - public WindowingInternals windowingInternals() { - return null; - } + @Override + public void mapPartition( + Iterable> values, + Collector> out) throws Exception { + + FlinkProcessContext context = new FlinkMultiOutputProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + outputMap, + sideInputs); - @Override - public PipelineOptions getPipelineOptions() { - return serializedPipelineOptions.getPipelineOptions(); - } + this.doFn.startBundle(context); - @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())); + if (!requiresWindowAccess || hasSideInputs) { + // we don't need to explode the windows + for (WindowedValue value : values) { + context = context.forWindowedValue(value); + doFn.processElement(context); } - 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)); + } else { + // we need to explode the windows because we have per-window + // side inputs and window access also only works if an element + // is in only one window + for (WindowedValue value : values) { + for (WindowedValue explodedValue: value.explodeWindows()) { + context = context.forWindowedValue(value); + doFn.processElement(context); + } } } - @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; - } + this.doFn.finishBundle(context); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java new file mode 100644 index 000000000000..71b6d27ddba0 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.translation.functions; + +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.join.RawUnionValue; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.util.Collection; +import java.util.Map; + +/** + * {@link DoFn.ProcessContext} for {@link FlinkMultiOutputDoFnFunction} that supports + * side outputs. + */ +class FlinkMultiOutputProcessContext + extends FlinkProcessContext { + + // we need a different Collector from the base class + private final Collector> collector; + + private final Map, Integer> outputMap; + + + FlinkMultiOutputProcessContext( + PipelineOptions pipelineOptions, + RuntimeContext runtimeContext, + DoFn doFn, + WindowingStrategy windowingStrategy, + Collector> collector, + Map, Integer> outputMap, + Map, WindowingStrategy> sideInputs) { + super( + pipelineOptions, + runtimeContext, + doFn, + windowingStrategy, + new Collector>() { + @Override + public void collect(WindowedValue outputTWindowedValue) { + + } + + @Override + public void close() { + + } + }, + sideInputs); + + this.collector = collector; + this.outputMap = outputMap; + } + + @Override + public FlinkProcessContext forWindowedValue( + WindowedValue windowedValue) { + this.windowedValue = windowedValue; + return this; + } + + @Override + public void outputWithTimestamp(OutputT value, Instant timestamp) { + if (windowedValue == null) { + // we are in startBundle() or finishBundle() + + try { + Collection windows = windowingStrategy.getWindowFn().assignWindows( + new FlinkNoElementAssignContext( + windowingStrategy.getWindowFn(), + value, + timestamp)); + + collector.collect( + WindowedValue.of( + new RawUnionValue(0, value), + timestamp != null ? timestamp : new Instant(Long.MIN_VALUE), + windows, + PaneInfo.NO_FIRING)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + collector.collect( + WindowedValue.of( + new RawUnionValue(0, value), + windowedValue.getTimestamp(), + windowedValue.getWindows(), + windowedValue.getPane())); + } + } + + @Override + protected void outputWithTimestampAndWindow( + OutputT value, + Instant timestamp, + Collection windows, + PaneInfo pane) { + collector.collect( + WindowedValue.of( + new RawUnionValue(0, value), timestamp, windows, pane)); + } + + @Override + @SuppressWarnings("unchecked") + public void sideOutput(TupleTag tag, T value) { + if (windowedValue != null) { + sideOutputWithTimestamp(tag, value, windowedValue.getTimestamp()); + } else { + sideOutputWithTimestamp(tag, value, null); + } + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T value, Instant timestamp) { + Integer index = outputMap.get(tag); + + if (index == null) { + throw new IllegalArgumentException("Unknown side output tag: " + tag); + } + + if (windowedValue == null) { + // we are in startBundle() or finishBundle() + + try { + Collection windows = windowingStrategy.getWindowFn().assignWindows( + new FlinkNoElementAssignContext( + windowingStrategy.getWindowFn(), + value, + timestamp)); + + collector.collect( + WindowedValue.of( + new RawUnionValue(index, value), + timestamp != null ? timestamp : new Instant(Long.MIN_VALUE), + windows, + PaneInfo.NO_FIRING)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + collector.collect( + WindowedValue.of( + new RawUnionValue(index, value), + windowedValue.getTimestamp(), + windowedValue.getWindows(), + windowedValue.getPane())); + } + + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java index 58a36b27c5dd..9205a5520f82 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java @@ -18,27 +18,34 @@ package org.apache.beam.runners.flink.translation.functions; import org.apache.beam.sdk.transforms.join.RawUnionValue; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; /** - * A FlatMap function that filters out those elements that don't belong in this output. We need - * this to implement MultiOutput ParDo functions. + * A {@link FlatMapFunction} function that filters out those elements that don't belong in this + * output. We need this to implement MultiOutput ParDo functions in combination with + * {@link FlinkMultiOutputDoFnFunction}. */ -public class FlinkMultiOutputPruningFunction implements FlatMapFunction { +public class FlinkMultiOutputPruningFunction + implements FlatMapFunction, WindowedValue> { - private final int outputTag; + private final int ourOutputTag; - public FlinkMultiOutputPruningFunction(int outputTag) { - this.outputTag = outputTag; + public FlinkMultiOutputPruningFunction(int ourOutputTag) { + this.ourOutputTag = ourOutputTag; } @Override @SuppressWarnings("unchecked") - public void flatMap(RawUnionValue rawUnionValue, Collector collector) throws Exception { - if (rawUnionValue.getUnionTag() == outputTag) { - collector.collect((T) rawUnionValue.getValue()); + public void flatMap( + WindowedValue windowedValue, + Collector> collector) throws Exception { + int unionTag = windowedValue.getValue().getUnionTag(); + if (unionTag == ourOutputTag) { + collector.collect( + (WindowedValue) windowedValue.withValue(windowedValue.getValue().getValue())); } } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java new file mode 100644 index 000000000000..892f7a1f33f0 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.translation.functions; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; + +import org.joda.time.Instant; + +import java.util.Collection; + +/** + * {@link WindowFn.AssignContext} for calling a {@link WindowFn} for elements emitted from + * {@link org.apache.beam.sdk.transforms.DoFn#startBundle(DoFn.Context)} + * or {@link DoFn#finishBundle(DoFn.Context)}. + * + *

In those cases the {@code WindowFn} is not allowed to access any element information. + */ +class FlinkNoElementAssignContext + extends WindowFn.AssignContext { + + private final InputT element; + private final Instant timestamp; + + FlinkNoElementAssignContext( + WindowFn fn, + InputT element, + Instant timestamp) { + fn.super(); + + this.element = element; + // the timestamp can be null, in that case output is called + // without a timestamp + this.timestamp = timestamp; + } + + @Override + public InputT element() { + return element; + } + + @Override + public Instant timestamp() { + if (timestamp != null) { + return timestamp; + } else { + throw new UnsupportedOperationException("No timestamp available."); + } + } + + @Override + public Collection windows() { + throw new UnsupportedOperationException("No windows available."); + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index a2bab2b3060f..c29e1df2ceb0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -17,45 +17,170 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.CombineFnBase; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; -import org.apache.flink.api.common.functions.GroupCombineFunction; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import org.apache.flink.api.common.functions.RichGroupCombineFunction; import org.apache.flink.util.Collector; +import org.joda.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; import java.util.Iterator; +import java.util.Map; /** - * Flink {@link org.apache.flink.api.common.functions.GroupCombineFunction} for executing a - * {@link org.apache.beam.sdk.transforms.Combine.PerKey} operation. This reads the input - * {@link org.apache.beam.sdk.values.KV} elements VI, extracts the key and emits accumulated - * values which have the intermediate format VA. + * This is is the first step for executing a {@link org.apache.beam.sdk.transforms.Combine.PerKey} + * on Flink. The second part is {@link FlinkReduceFunction}. This function performs a local + * combine step before shuffling while the latter does the final combination after a shuffle. + * + *

The input to {@link #combine(Iterable, Collector)} are elements of the same key but + * for different windows. We have to ensure that we only combine elements of matching + * windows. */ -public class FlinkPartialReduceFunction implements GroupCombineFunction, KV> { +public class FlinkPartialReduceFunction + extends RichGroupCombineFunction>, WindowedValue>> { + + protected final CombineFnBase.PerKeyCombineFn combineFn; + + protected final DoFn, KV> doFn; + + protected final WindowingStrategy windowingStrategy; + + protected final SerializedPipelineOptions serializedOptions; - private final Combine.KeyedCombineFn keyedCombineFn; + protected final Map, WindowingStrategy> sideInputs; - public FlinkPartialReduceFunction(Combine.KeyedCombineFn - keyedCombineFn) { - this.keyedCombineFn = keyedCombineFn; + public FlinkPartialReduceFunction( + CombineFnBase.PerKeyCombineFn combineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { + + this.combineFn = combineFn; + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + + // dummy DoFn because we need one for ProcessContext + this.doFn = new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) throws Exception { + + } + }; } @Override - public void combine(Iterable> elements, Collector> out) throws Exception { + public void combine( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); + + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) windowingStrategy.getOutputTimeFn(); + + // get all elements so that we can sort them, has to fit into + // memory + // this seems very unprudent, but correct, for now + ArrayList>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + sortedInput.add(exploded); + } + } + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> o2) { + return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp() + .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp()); + } + }); + + // iterate over the elements that are sorted by window timestamp + // + final Iterator>> iterator = sortedInput.iterator(); - 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); + WindowedValue> currentValue = iterator.next(); + K key = currentValue.getValue().getKey(); + BoundedWindow currentWindow = Iterables.getFirst(currentValue.getWindows(), null); + InputT firstValue = currentValue.getValue().getValue(); + processContext = processContext.forWindowedValue(currentValue); + AccumT accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, firstValue, processContext); + + // we use this to keep track of the timestamps assigned by the OutputTimeFn + Instant windowTimestamp = + outputTimeFn.assignOutputTime(currentValue.getTimestamp(), currentWindow); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + BoundedWindow nextWindow = Iterables.getOnlyElement(nextValue.getWindows()); + + if (nextWindow.equals(currentWindow)) { + // continue accumulating + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + + windowTimestamp = outputTimeFn.combine( + windowTimestamp, + outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow)); + + } else { + // emit the value that we currently have + out.collect( + WindowedValue.of( + KV.of(key, accumulator), + windowTimestamp, + currentWindow, + PaneInfo.NO_FIRING)); + + currentWindow = nextWindow; + InputT value = nextValue.getValue().getValue(); + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.createAccumulator(key, processContext); + accumulator = combineFnRunner.addInput(key, accumulator, value, processContext); + windowTimestamp = outputTimeFn.assignOutputTime(nextValue.getTimestamp(), currentWindow); + } } - out.collect(KV.of(key, accumulator)); + // emit the final accumulator + out.collect( + WindowedValue.of( + KV.of(key, accumulator), + windowTimestamp, + currentWindow, + PaneInfo.NO_FIRING)); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java new file mode 100644 index 000000000000..0f1885ca5192 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java @@ -0,0 +1,324 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.translation.functions; + +import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.util.Collector; +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; + +/** + * {@link org.apache.beam.sdk.transforms.DoFn.ProcessContext} for our Flink Wrappers. + */ +class FlinkProcessContext + extends DoFn.ProcessContext { + + private final PipelineOptions pipelineOptions; + private final RuntimeContext runtimeContext; + private Collector> collector; + private final boolean requiresWindowAccess; + + protected WindowedValue windowedValue; + + protected WindowingStrategy windowingStrategy; + + private final Map, WindowingStrategy> sideInputs; + + FlinkProcessContext( + PipelineOptions pipelineOptions, + RuntimeContext runtimeContext, + DoFn doFn, + WindowingStrategy windowingStrategy, + Collector> collector, + Map, WindowingStrategy> sideInputs) { + doFn.super(); + Preconditions.checkNotNull(pipelineOptions); + Preconditions.checkNotNull(runtimeContext); + Preconditions.checkNotNull(doFn); + Preconditions.checkNotNull(collector); + + this.pipelineOptions = pipelineOptions; + this.runtimeContext = runtimeContext; + this.collector = collector; + this.requiresWindowAccess = doFn instanceof DoFn.RequiresWindowAccess; + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + + super.setupDelegateAggregators(); + } + + FlinkProcessContext( + PipelineOptions pipelineOptions, + RuntimeContext runtimeContext, + DoFn doFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs) { + doFn.super(); + Preconditions.checkNotNull(pipelineOptions); + Preconditions.checkNotNull(runtimeContext); + Preconditions.checkNotNull(doFn); + + this.pipelineOptions = pipelineOptions; + this.runtimeContext = runtimeContext; + this.collector = null; + this.requiresWindowAccess = doFn instanceof DoFn.RequiresWindowAccess; + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + + super.setupDelegateAggregators(); + } + + public FlinkProcessContext forOutput( + Collector> collector) { + this.collector = collector; + + // for now, returns ourselves, to be easy on the GC + return this; + } + + + + public FlinkProcessContext forWindowedValue( + WindowedValue windowedValue) { + this.windowedValue = windowedValue; + + // for now, returns ourselves, to be easy on the GC + return this; + } + + @Override + public InputT element() { + return this.windowedValue.getValue(); + } + + + @Override + public Instant timestamp() { + return windowedValue.getTimestamp(); + } + + @Override + public BoundedWindow window() { + if (!requiresWindowAccess) { + throw new UnsupportedOperationException( + "window() is only available in the context of a DoFn marked as RequiresWindow."); + } + return Iterables.getOnlyElement(windowedValue.getWindows()); + } + + @Override + public PaneInfo pane() { + return windowedValue.getPane(); + } + + @Override + public WindowingInternals windowingInternals() { + + return new WindowingInternals() { + + @Override + public StateInternals stateInternals() { + throw new UnsupportedOperationException(); + } + + @Override + public void outputWindowedValue( + OutputT value, + Instant timestamp, + Collection windows, + PaneInfo pane) { + collector.collect(WindowedValue.of(value, timestamp, windows, pane)); + outputWithTimestampAndWindow(value, timestamp, windows, pane); + } + + @Override + public TimerInternals timerInternals() { + throw new UnsupportedOperationException(); + } + + @Override + public Collection windows() { + return windowedValue.getWindows(); + } + + @Override + public PaneInfo pane() { + return windowedValue.getPane(); + } + + @Override + public void writePCollectionViewData(TupleTag tag, + Iterable> data, Coder elemCoder) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public ViewT sideInput( + PCollectionView view, + BoundedWindow mainInputWindow) { + + Preconditions.checkNotNull(view, "View passed to sideInput cannot be null"); + Preconditions.checkNotNull( + sideInputs.get(view), + "Side input for " + view + " not available."); + + // get the side input strategy for mapping the window + WindowingStrategy windowingStrategy = sideInputs.get(view); + + BoundedWindow sideInputWindow = + windowingStrategy.getWindowFn().getSideInputWindow(mainInputWindow); + + Map sideInputs = + runtimeContext.getBroadcastVariableWithInitializer( + view.getTagInternal().getId(), new SideInputInitializer<>(view)); + return sideInputs.get(sideInputWindow); + } + }; + } + + @Override + public PipelineOptions getPipelineOptions() { + return pipelineOptions; + } + + @Override + public ViewT sideInput(PCollectionView view) { + Preconditions.checkNotNull(view, "View passed to sideInput cannot be null"); + Preconditions.checkNotNull(sideInputs.get(view), "Side input for " + view + " not available."); + Iterator windowIter = windowedValue.getWindows().iterator(); + BoundedWindow window; + if (!windowIter.hasNext()) { + throw new IllegalStateException( + "sideInput called when main input element is not in any windows"); + } else { + window = windowIter.next(); + if (windowIter.hasNext()) { + throw new IllegalStateException( + "sideInput called when main input element is in multiple windows"); + } + } + + // get the side input strategy for mapping the window + WindowingStrategy windowingStrategy = sideInputs.get(view); + + BoundedWindow sideInputWindow = + windowingStrategy.getWindowFn().getSideInputWindow(window); + + Map sideInputs = + runtimeContext.getBroadcastVariableWithInitializer( + view.getTagInternal().getId(), new SideInputInitializer<>(view)); + ViewT result = sideInputs.get(sideInputWindow); + if (result == null) { + result = view.fromIterableInternal(Collections.>emptyList()); + } + return result; + } + + @Override + public void output(OutputT value) { + if (windowedValue != null) { + outputWithTimestamp(value, windowedValue.getTimestamp()); + } else { + outputWithTimestamp(value, null); + } + } + + @Override + public void outputWithTimestamp(OutputT value, Instant timestamp) { + if (windowedValue == null) { + // we are in startBundle() or finishBundle() + + try { + Collection windows = windowingStrategy.getWindowFn().assignWindows( + new FlinkNoElementAssignContext( + windowingStrategy.getWindowFn(), + value, + timestamp)); + + collector.collect( + WindowedValue.of( + value, + timestamp != null ? timestamp : new Instant(Long.MIN_VALUE), + windows, + PaneInfo.NO_FIRING)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + collector.collect( + WindowedValue.of( + value, + timestamp, + windowedValue.getWindows(), + windowedValue.getPane())); + } + } + + protected void outputWithTimestampAndWindow( + OutputT value, + Instant timestamp, + Collection windows, + PaneInfo pane) { + collector.collect( + WindowedValue.of( + value, timestamp, windows, pane)); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + throw new UnsupportedOperationException(); + } + + @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); + runtimeContext.addAccumulator(name, wrapper); + return wrapper; + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java index 43e458fc3720..9cbc6b914765 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java @@ -17,43 +17,179 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.CombineFnBase; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.PerKeyCombineFnRunner; +import org.apache.beam.sdk.util.PerKeyCombineFnRunners; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; -import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.util.Collector; +import org.joda.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; import java.util.Iterator; +import java.util.List; +import java.util.Map; /** - * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a - * {@link org.apache.beam.sdk.transforms.Combine.PerKey} operation. This reads the input - * {@link org.apache.beam.sdk.values.KV} elements, extracts the key and merges the - * accumulators resulting from the PartialReduce which produced the input VA. + * This is the second part for executing a {@link org.apache.beam.sdk.transforms.Combine.PerKey} + * on Flink, the second part is {@link FlinkReduceFunction}. This function performs the final + * combination of the pre-combined values after a shuffle. + * + *

The input to {@link #reduce(Iterable, Collector)} are elements of the same key but + * for different windows. We have to ensure that we only combine elements of matching + * windows. */ -public class FlinkReduceFunction implements GroupReduceFunction, KV> { +public class FlinkReduceFunction + extends RichGroupReduceFunction>, WindowedValue>> { + + protected final CombineFnBase.PerKeyCombineFn combineFn; + + protected final DoFn, KV> doFn; + + protected final WindowingStrategy windowingStrategy; + + protected final Map, WindowingStrategy> sideInputs; + + protected final SerializedPipelineOptions serializedOptions; - private final Combine.KeyedCombineFn keyedCombineFn; + public FlinkReduceFunction( + CombineFnBase.PerKeyCombineFn keyedCombineFn, + WindowingStrategy windowingStrategy, + Map, WindowingStrategy> sideInputs, + PipelineOptions pipelineOptions) { - public FlinkReduceFunction(Combine.KeyedCombineFn keyedCombineFn) { - this.keyedCombineFn = keyedCombineFn; + this.combineFn = keyedCombineFn; + + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + + this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + + // dummy DoFn because we need one for ProcessContext + this.doFn = new DoFn, KV>() { + @Override + public void processElement(ProcessContext c) throws Exception { + + } + }; } @Override - public void reduce(Iterable> values, Collector> out) throws Exception { - Iterator> it = values.iterator(); + public void reduce( + Iterable>> elements, + Collector>> out) throws Exception { + + FlinkProcessContext, KV> processContext = + new FlinkProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + sideInputs); + + PerKeyCombineFnRunner combineFnRunner = + PerKeyCombineFnRunners.create(combineFn); - KV current = it.next(); - K k = current.getKey(); - VA accumulator = current.getValue(); + @SuppressWarnings("unchecked") + OutputTimeFn outputTimeFn = + (OutputTimeFn) windowingStrategy.getOutputTimeFn(); - while (it.hasNext()) { - current = it.next(); - keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) ); + + // get all elements so that we can sort them, has to fit into + // memory + // this seems very unprudent, but correct, for now + ArrayList>> sortedInput = Lists.newArrayList(); + for (WindowedValue> inputValue: elements) { + for (WindowedValue> exploded: inputValue.explodeWindows()) { + sortedInput.add(exploded); + } + } + Collections.sort(sortedInput, new Comparator>>() { + @Override + public int compare( + WindowedValue> o1, + WindowedValue> o2) { + return Iterables.getOnlyElement(o1.getWindows()).maxTimestamp() + .compareTo(Iterables.getOnlyElement(o2.getWindows()).maxTimestamp()); + } + }); + + // iterate over the elements that are sorted by window timestamp + // + final Iterator>> iterator = sortedInput.iterator(); + + // get the first accumulator + WindowedValue> currentValue = iterator.next(); + K key = currentValue.getValue().getKey(); + BoundedWindow currentWindow = Iterables.getFirst(currentValue.getWindows(), null); + AccumT accumulator = currentValue.getValue().getValue(); + + // we use this to keep track of the timestamps assigned by the OutputTimeFn, + // in FlinkPartialReduceFunction we already merge the timestamps assigned + // to individual elements, here we just merge them + List windowTimestamps = new ArrayList<>(); + windowTimestamps.add(currentValue.getTimestamp()); + + while (iterator.hasNext()) { + WindowedValue> nextValue = iterator.next(); + BoundedWindow nextWindow = Iterables.getOnlyElement(nextValue.getWindows()); + + if (nextWindow.equals(currentWindow)) { + // continue accumulating + processContext = processContext.forWindowedValue(nextValue); + accumulator = combineFnRunner.mergeAccumulators( + key, ImmutableList.of(accumulator, nextValue.getValue().getValue()), processContext); + + windowTimestamps.add(nextValue.getTimestamp()); + } else { + // emit the value that we currently have + processContext = processContext.forWindowedValue(currentValue); + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + outputTimeFn.merge(currentWindow, windowTimestamps), + currentWindow, + PaneInfo.NO_FIRING)); + + windowTimestamps.clear(); + + currentWindow = nextWindow; + accumulator = nextValue.getValue().getValue(); + windowTimestamps.add(nextValue.getTimestamp()); + } + + // we have to keep track so that we can set the context to the right + // windowed value when windows change in the iterable + currentValue = nextValue; } - out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator))); + // if at the end of the iteration we have a change in windows + // the ProcessContext will not have been updated + processContext = processContext.forWindowedValue(currentValue); + + // emit the final accumulator + out.collect( + WindowedValue.of( + KV.of(key, combineFnRunner.extractOutput(key, accumulator, processContext)), + outputTimeFn.merge(currentWindow, windowTimestamps), + currentWindow, + PaneInfo.NO_FIRING)); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java new file mode 100644 index 000000000000..451b31b12c5e --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.flink.translation.functions; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollectionView; + +import org.apache.flink.api.common.functions.BroadcastVariableInitializer; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * {@link BroadcastVariableInitializer} that initializes the broadcast input as a {@code Map} + * from window to side input. + */ +public class SideInputInitializer + implements BroadcastVariableInitializer, Map> { + + PCollectionView view; + + public SideInputInitializer(PCollectionView view) { + this.view = view; + } + + @Override + public Map initializeBroadcastVariable( + Iterable> inputValues) { + + // first partition into windows + Map>> partitionedElements = new HashMap<>(); + for (WindowedValue value: inputValues) { + for (BoundedWindow window: value.getWindows()) { + List> windowedValues = partitionedElements.get(window); + if (windowedValues == null) { + windowedValues = new ArrayList<>(); + partitionedElements.put(window, windowedValues); + } + windowedValues.add(value); + } + } + + Map resultMap = new HashMap<>(); + + for (Map.Entry>> elements: + partitionedElements.entrySet()) { + + @SuppressWarnings("unchecked") + Iterable> elementsIterable = + (List>) (List) elements.getValue(); + + resultMap.put(elements.getKey(), view.fromIterableInternal(elementsIterable)); + } + + return resultMap; + } +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java deleted file mode 100644 index cc6fd8b70917..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.functions; - - -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.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; -import java.util.List; - -/** - * A UnionCoder encodes RawUnionValues. - * - * This file copied from {@link org.apache.beam.sdk.transforms.join.UnionCoder} - */ -@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); - } -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java index 895ecef1b92e..4434cf8726e1 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java @@ -18,7 +18,8 @@ package org.apache.beam.runners.flink.translation.types; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.util.WindowedValue; import com.google.common.base.Preconditions; @@ -71,9 +72,6 @@ public boolean isKeyType() { @Override @SuppressWarnings("unchecked") public TypeSerializer createSerializer(ExecutionConfig config) { - if (coder instanceof VoidCoder) { - return (TypeSerializer) new VoidCoderTypeSerializer(); - } return new CoderTypeSerializer<>(coder); } @@ -84,8 +82,12 @@ public int getTotalFields() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } CoderTypeInformation that = (CoderTypeInformation) o; @@ -113,6 +115,11 @@ public String toString() { @Override public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) { - return new CoderComparator<>(coder); + WindowedValue.WindowedValueCoder windowCoder = (WindowedValue.WindowedValueCoder) coder; + if (windowCoder.getValueCoder() instanceof KvCoder) { + return new KvCoderComperator(windowCoder); + } else { + return new CoderComparator<>(coder); + } } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java index c6f3921971a6..097316b242fd 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java @@ -33,7 +33,7 @@ /** * Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for - * Dataflow {@link org.apache.beam.sdk.coders.Coder}s + * Dataflow {@link org.apache.beam.sdk.coders.Coder Coders}. */ public class CoderTypeSerializer extends TypeSerializer { @@ -128,14 +128,20 @@ public T deserialize(T t, DataInputView dataInputView) throws IOException { } @Override - public void copy(DataInputView dataInputView, DataOutputView dataOutputView) throws IOException { + 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; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } CoderTypeSerializer that = (CoderTypeSerializer) o; return coder.equals(that.coder); diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java index 6f0c651406a2..79b127d1062c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java @@ -20,6 +20,8 @@ import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -31,14 +33,13 @@ import java.io.ObjectInputStream; /** - * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for - * {@link org.apache.beam.sdk.coders.KvCoder}. We have a special comparator + * Flink {@link TypeComparator} for {@link KvCoder}. We have a special comparator * for {@link KV} that always compares on the key only. */ -public class KvCoderComperator extends TypeComparator> { +public class KvCoderComperator extends TypeComparator>> { - private KvCoder coder; - private Coder keyCoder; + private final WindowedValue.WindowedValueCoder> coder; + private final Coder keyCoder; // We use these for internal encoding/decoding for creating copies and comparing // serialized forms using a Coder @@ -52,9 +53,10 @@ public class KvCoderComperator extends TypeComparator> { // For deserializing the key private transient DataInputViewWrapper inputWrapper; - public KvCoderComperator(KvCoder coder) { + public KvCoderComperator(WindowedValue.WindowedValueCoder> coder) { this.coder = coder; - this.keyCoder = coder.getKeyCoder(); + KvCoder kvCoder = (KvCoder) coder.getValueCoder(); + this.keyCoder = kvCoder.getKeyCoder(); buffer1 = new InspectableByteArrayOutputStream(); buffer2 = new InspectableByteArrayOutputStream(); @@ -74,8 +76,8 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE } @Override - public int hash(KV record) { - K key = record.getKey(); + public int hash(WindowedValue> record) { + K key = record.getValue().getKey(); if (key != null) { return key.hashCode(); } else { @@ -84,27 +86,27 @@ public int hash(KV record) { } @Override - public void setReference(KV toCompare) { + public void setReference(WindowedValue> toCompare) { referenceBuffer.reset(); try { - keyCoder.encode(toCompare.getKey(), referenceBuffer, Coder.Context.OUTER); + keyCoder.encode(toCompare.getValue().getKey(), referenceBuffer, Coder.Context.OUTER); } catch (IOException e) { throw new RuntimeException("Could not set reference " + toCompare + ": " + e); } } @Override - public boolean equalToReference(KV candidate) { + public boolean equalToReference(WindowedValue> candidate) { try { buffer2.reset(); - keyCoder.encode(candidate.getKey(), buffer2, Coder.Context.OUTER); + keyCoder.encode(candidate.getValue().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++ ) { + for (int i = 0; i < len; i++) { if (arr[i] != arrOther[i]) { return false; } @@ -116,8 +118,9 @@ public boolean equalToReference(KV candidate) { } @Override - public int compareToReference(TypeComparator> other) { - InspectableByteArrayOutputStream otherReferenceBuffer = ((KvCoderComperator) other).referenceBuffer; + public int compareToReference(TypeComparator>> other) { + InspectableByteArrayOutputStream otherReferenceBuffer = + ((KvCoderComperator) other).referenceBuffer; byte[] arr = referenceBuffer.getBuffer(); byte[] arrOther = otherReferenceBuffer.getBuffer(); @@ -135,19 +138,19 @@ public int compareToReference(TypeComparator> other) { @Override - public int compare(KV first, KV second) { + public int compare(WindowedValue> first, WindowedValue> second) { try { buffer1.reset(); buffer2.reset(); - keyCoder.encode(first.getKey(), buffer1, Coder.Context.OUTER); - keyCoder.encode(second.getKey(), buffer2, Coder.Context.OUTER); + keyCoder.encode(first.getValue().getKey(), buffer1, Coder.Context.OUTER); + keyCoder.encode(second.getValue().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++ ) { + for (int i = 0; i < len; i++) { if (arr[i] != arrOther[i]) { return arr[i] - arrOther[i]; } @@ -159,38 +162,19 @@ public int compare(KV first, KV second) { } @Override - public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException { - + public int compareSerialized( + DataInputView firstSource, + DataInputView secondSource) throws IOException { inputWrapper.setInputView(firstSource); - K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED); + WindowedValue> first = coder.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); - } + WindowedValue> second = coder.decode(inputWrapper, Coder.Context.NESTED); + return compare(first, second); } @Override public boolean supportsNormalizedKey() { - return true; + return false; } @Override @@ -209,12 +193,18 @@ public boolean isNormalizedKeyPrefixOnly(int keyBytes) { } @Override - public void putNormalizedKey(KV record, MemorySegment target, int offset, int numBytes) { + public void putNormalizedKey( + WindowedValue> record, + MemorySegment target, + int offset, + int numBytes) { + buffer1.reset(); try { - keyCoder.encode(record.getKey(), buffer1, Coder.Context.NESTED); + keyCoder.encode(record.getValue().getKey(), buffer1, Coder.Context.NESTED); } catch (IOException e) { - throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e); + throw new RuntimeException( + "Could not serializer " + record + " using coder " + coder + ": " + e); } final byte[] data = buffer1.getBuffer(); final int limit = offset + numBytes; @@ -231,12 +221,16 @@ public void putNormalizedKey(KV record, MemorySegment target, int offset, } @Override - public void writeWithKeyNormalization(KV record, DataOutputView target) throws IOException { + public void writeWithKeyNormalization( + WindowedValue> record, + DataOutputView target) throws IOException { throw new UnsupportedOperationException(); } @Override - public KV readWithKeyDenormalization(KV reuse, DataInputView source) throws IOException { + public WindowedValue> readWithKeyDenormalization( + WindowedValue> reuse, + DataInputView source) throws IOException { throw new UnsupportedOperationException(); } @@ -246,14 +240,14 @@ public boolean invertNormalizedKey() { } @Override - public TypeComparator> duplicate() { + 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(); + WindowedValue> kv = (WindowedValue>) record; + K k = kv.getValue().getKey(); target[index] = k; return 1; } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java index 74f3821dfb2f..ba53f640bb81 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.flink.translation.types; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import com.google.common.base.Preconditions; @@ -31,27 +32,32 @@ import java.util.List; /** - * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for - * Dataflow {@link org.apache.beam.sdk.coders.KvCoder}. + * Flink {@link TypeInformation} for {@link KvCoder}. This creates special comparator + * for {@link KV} that always compares on the key only. */ -public class KvCoderTypeInformation extends CompositeType> { +public class KvCoderTypeInformation extends CompositeType>> { - private KvCoder coder; + private final WindowedValue.WindowedValueCoder> 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(); + private static Object dummy = new Object(); @SuppressWarnings("unchecked") - public KvCoderTypeInformation(KvCoder coder) { - super(((Class>) DUMMY.getClass())); + public KvCoderTypeInformation(WindowedValue.WindowedValueCoder> 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); + public TypeComparator>> createComparator( + int[] logicalKeyFields, + boolean[] orders, + int logicalFieldOffset, + ExecutionConfig config) { + return new KvCoderComperator(coder); } @Override @@ -71,7 +77,7 @@ public int getArity() { @Override @SuppressWarnings("unchecked") - public Class> getTypeClass() { + public Class>> getTypeClass() { return privateGetTypeClass(); } @@ -87,7 +93,7 @@ public boolean isKeyType() { @Override @SuppressWarnings("unchecked") - public TypeSerializer> createSerializer(ExecutionConfig config) { + public TypeSerializer>> createSerializer(ExecutionConfig config) { return new CoderTypeSerializer<>(coder); } @@ -98,8 +104,12 @@ public int getTotalFields() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } KvCoderTypeInformation that = (KvCoderTypeInformation) o; @@ -122,10 +132,11 @@ public String toString() { @Override @SuppressWarnings("unchecked") public TypeInformation getTypeAt(int pos) { + KvCoder kvCoder = (KvCoder) coder.getValueCoder(); if (pos == 0) { - return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); + return (TypeInformation) new CoderTypeInformation<>(kvCoder.getKeyCoder()); } else if (pos == 1) { - return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); + return (TypeInformation) new CoderTypeInformation<>(kvCoder.getValueCoder()); } else { throw new RuntimeException("Invalid field position " + pos); } @@ -134,11 +145,12 @@ public TypeInformation getTypeAt(int pos) { @Override @SuppressWarnings("unchecked") public TypeInformation getTypeAt(String fieldExpression) { + KvCoder kvCoder = (KvCoder) coder.getValueCoder(); switch (fieldExpression) { case "key": - return (TypeInformation) new CoderTypeInformation<>(coder.getKeyCoder()); + return (TypeInformation) new CoderTypeInformation<>(kvCoder.getKeyCoder()); case "value": - return (TypeInformation) new CoderTypeInformation<>(coder.getValueCoder()); + return (TypeInformation) new CoderTypeInformation<>(kvCoder.getValueCoder()); default: throw new UnsupportedOperationException("Only KvCoder has fields."); } @@ -162,17 +174,24 @@ public int getFieldIndex(String fieldName) { } @Override - public void getFlatFields(String fieldExpression, int offset, List result) { - CoderTypeInformation keyTypeInfo = new CoderTypeInformation<>(coder.getKeyCoder()); + public void getFlatFields( + String fieldExpression, + int offset, + List result) { + KvCoder kvCoder = (KvCoder) coder.getValueCoder(); + + CoderTypeInformation keyTypeInfo = + new CoderTypeInformation<>(kvCoder.getKeyCoder()); result.add(new FlatFieldDescriptor(0, keyTypeInfo)); } @Override - protected TypeComparatorBuilder> createTypeComparatorBuilder() { + protected TypeComparatorBuilder>> createTypeComparatorBuilder() { return new KvCoderTypeComparatorBuilder(); } - private class KvCoderTypeComparatorBuilder implements TypeComparatorBuilder> { + private class KvCoderTypeComparatorBuilder + implements TypeComparatorBuilder>> { @Override public void initializeTypeComparatorBuilder(int size) {} @@ -181,7 +200,7 @@ public void initializeTypeComparatorBuilder(int size) {} public void addComparatorField(int fieldId, TypeComparator comparator) {} @Override - public TypeComparator> createTypeComparator(ExecutionConfig config) { + public TypeComparator>> createTypeComparator(ExecutionConfig config) { return new KvCoderComperator<>(coder); } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java deleted file mode 100644 index 7b48208845fd..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.types; - -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; - -/** - * Special Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for - * {@link org.apache.beam.sdk.coders.VoidCoder}. We need this because Flink does not - * allow returning {@code null} from an input reader. We return a {@link VoidValue} instead - * that behaves like a {@code null}, hopefully. - */ -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(); - } - -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java deleted file mode 100644 index e5567d3ea3b2..000000000000 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.translation.wrappers; - -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; - -import com.google.common.collect.Lists; - -import org.apache.flink.api.common.accumulators.Accumulator; - -import java.io.Serializable; - -/** - * Wrapper that wraps a {@link org.apache.beam.sdk.transforms.Combine.CombineFn} - * in a Flink {@link org.apache.flink.api.common.accumulators.Accumulator} for using - * the combine function as an aggregator in a {@link org.apache.beam.sdk.transforms.ParDo} - * operation. - */ -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 Serializable getLocalValue() { - return (Serializable) 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 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 String getName() { - return "CombineFn: " + combiner.toString(); - } - - @Override - public Combine.CombineFn getCombineFn() { - return combiner; - } - -} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java index eb32fa2fd74a..82d3fb8ffae3 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -33,20 +33,21 @@ * the function as an aggregator in a {@link org.apache.beam.sdk.transforms.ParDo} * operation. */ -public class SerializableFnAggregatorWrapper implements Aggregator, Accumulator { +public class SerializableFnAggregatorWrapper + implements Aggregator, Accumulator { - private AO aa; - private Combine.CombineFn combiner; + private OutputT aa; + private Combine.CombineFn combiner; - public SerializableFnAggregatorWrapper(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 void add(InputT value) { + this.aa = combiner.apply(ImmutableList.of((InputT) aa, value)); } @Override @@ -56,17 +57,17 @@ public Serializable getLocalValue() { @Override public void resetLocal() { - this.aa = combiner.apply(ImmutableList.of()); + 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())); + public void merge(Accumulator other) { + this.aa = combiner.apply(ImmutableList.of((InputT) aa, (InputT) other.getLocalValue())); } @Override - public void addValue(AI value) { + public void addValue(InputT value) { add(value); } @@ -76,15 +77,15 @@ public String getName() { } @Override - public Combine.CombineFn getCombineFn() { + public Combine.CombineFn getCombineFn() { return combiner; } @Override - public Accumulator clone() { + public Accumulator clone() { // copy it by merging - AO resultCopy = combiner.apply(Lists.newArrayList((AI) aa)); - SerializableFnAggregatorWrapper result = new + OutputT resultCopy = combiner.apply(Lists.newArrayList((InputT) aa)); + SerializableFnAggregatorWrapper result = new SerializableFnAggregatorWrapper<>(combiner); result.aa = resultCopy; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java index 2766a873e35f..28a4b9ee59df 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java @@ -22,6 +22,7 @@ import org.apache.beam.sdk.io.Sink; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.configuration.Configuration; @@ -31,10 +32,11 @@ import java.lang.reflect.Field; /** - * Wrapper class to use generic Write.Bound transforms as sinks. + * Wrapper for executing a {@link Sink} on Flink as an {@link OutputFormat}. + * * @param The type of the incoming records. */ -public class SinkOutputFormat implements OutputFormat { +public class SinkOutputFormat implements OutputFormat> { private final Sink sink; @@ -88,9 +90,9 @@ public void open(int taskNumber, int numTasks) throws IOException { } @Override - public void writeRecord(T record) throws IOException { + public void writeRecord(WindowedValue record) throws IOException { try { - writer.write(record); + writer.write(record.getValue()); } catch (Exception e) { throw new IOException("Couldn't write record.", e); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java index debd1a14d525..1d06b1ac2fc9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java @@ -21,12 +21,16 @@ import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.InputSplitAssigner; +import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,10 +39,10 @@ /** - * A Flink {@link org.apache.flink.api.common.io.InputFormat} that wraps a - * Dataflow {@link org.apache.beam.sdk.io.Source}. + * Wrapper for executing a {@link Source} as a Flink {@link InputFormat}. */ -public class SourceInputFormat implements InputFormat> { +public class SourceInputFormat + implements InputFormat, SourceInputSplit> { private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class); private final BoundedSource initialSource; @@ -122,12 +126,16 @@ public boolean reachedEnd() throws IOException { } @Override - public T nextRecord(T t) throws IOException { + public WindowedValue nextRecord(WindowedValue t) throws IOException { if (inputAvailable) { final T current = reader.getCurrent(); + final Instant timestamp = reader.getCurrentTimestamp(); // advance reader to have a record ready next time inputAvailable = reader.advance(); - return current; + return WindowedValue.of( + current, + timestamp, + GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); } return null; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java index 8d9744fb15c2..0306aa1ac9d1 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; +import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; @@ -37,7 +38,6 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.SystemReduceFn; @@ -83,7 +83,7 @@ * 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 org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn}. + * ({@link org.apache.beam.runners.core.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 @@ -95,7 +95,7 @@ * 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 org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn} + * list, and are fed into the {@link org.apache.beam.runners.core.GroupAlsoByWindowsDoFn} * for furhter processing. */ public class FlinkGroupAlsoByWindowWrapper @@ -253,7 +253,7 @@ public void open() throws Exception { } /** - * Create the adequate {@link org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn}, + * Create the adequate {@link org.apache.beam.runners.core.GroupAlsoByWindowsDoFn}, * if not already created. * If a {@link org.apache.beam.sdk.transforms.Combine.KeyedCombineFn} was provided, then * a function with that combiner is created, so that elements are combined as they arrive. This is diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java index 3bf566bce762..6b69d547cf12 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; -import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; @@ -54,7 +53,7 @@ public static KeyedStream>, K> groupStreamByKey(Da @Override public K getKey(WindowedValue> value) throws Exception { - return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE : + return isKeyVoid ? (K) VoidValue.INSTANCE : value.getValue().getKey(); } @@ -64,4 +63,11 @@ public TypeInformation getProducedType() { } }); } + + // special type to return as key for null key + public static class VoidValue { + private VoidValue() {} + + public static VoidValue INSTANCE = new VoidValue(); + } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java index d6aff7d7a4ee..8cd8351021b4 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; -import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -47,17 +46,11 @@ public FlinkStreamingCreateFunction(List elements, 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.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } out.close(); diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java deleted file mode 100644 index 113fee0881de..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/AvroITCase.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.io.AvroIO; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; - -import com.google.common.base.Joiner; - -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(); - } - - private static class User { - - private String name; - private int favoriteNumber; - private String favoriteColor; - - public User() {} - - public User(String name, int favoriteNumber, String favoriteColor) { - this.name = name; - this.favoriteNumber = favoriteNumber; - this.favoriteColor = favoriteColor; - } - - public String getName() { - return name; - } - - public String getFavoriteColor() { - return favoriteColor; - } - - public int getFavoriteNumber() { - return favoriteNumber; - } - } - -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java deleted file mode 100644 index ac0a3d7d4d67..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.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.createForBatch(); - - 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/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java index f015a6680568..edde925c330c 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java @@ -60,7 +60,7 @@ public static FlinkTestPipeline createForStreaming() { * @return The Test Pipeline. */ private static FlinkTestPipeline create(boolean streaming) { - FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming); + TestFlinkPipelineRunner flinkRunner = TestFlinkPipelineRunner.create(streaming); return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions()); } diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java deleted file mode 100644 index 47685b6be6f3..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.runners.flink.util.JoinExamples; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableRow; -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.util.Arrays; -import java.util.List; - - -/** - * Unfortunately we need to copy the code from the Dataflow SDK because it is not public there. - */ -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(); - } -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java deleted file mode 100644 index 4d66fa421c5e..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.io.Serializable; - -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(); - } - -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java deleted file mode 100644 index a2ef4e29f403..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.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.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/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java index 66c959eea90c..bb79b270945c 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java @@ -28,6 +28,9 @@ import org.apache.flink.test.util.JavaProgramTestBase; +import java.io.File; +import java.net.URI; + /** * Reads from a bounded source in batch execution. */ @@ -44,6 +47,13 @@ public ReadSourceITCase(){ @Override protected void preSubmit() throws Exception { resultPath = getTempDirPath("result"); + + // need to create the dir, otherwise Beam sinks don't + // work for these tests + + if (!new File(new URI(resultPath)).mkdirs()) { + throw new RuntimeException("Could not create output dir."); + } } @Override @@ -56,7 +66,7 @@ protected void testProgram() throws Exception { runProgram(resultPath); } - private static void runProgram(String resultPath) { + private static void runProgram(String resultPath) throws Exception { Pipeline p = FlinkTestPipeline.createForBatch(); @@ -69,7 +79,7 @@ public void processElement(ProcessContext c) throws Exception { } })); - result.apply(TextIO.Write.to(resultPath)); + result.apply(TextIO.Write.to(new URI(resultPath).getPath() + "/part")); p.run(); } diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java deleted file mode 100644 index 471d3262a36c..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.RemoveDuplicates; -import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.util.Collections; -import java.util.List; - - -public class RemoveDuplicatesEmptyITCase extends JavaProgramTestBase { - - protected String resultPath; - - public RemoveDuplicatesEmptyITCase(){ - } - - static final String[] EXPECTED_RESULT = new String[] {}; - - @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 { - - List strings = Collections.emptyList(); - - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection input = - p.apply(Create.of(strings)) - .setCoder(StringUtf8Coder.of()); - - PCollection output = - input.apply(RemoveDuplicates.create()); - - output.apply(TextIO.Write.to(resultPath)); - p.run(); - } -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java deleted file mode 100644 index 0544f20eb310..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.RemoveDuplicates; -import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.util.Arrays; -import java.util.List; - - -public class RemoveDuplicatesITCase extends JavaProgramTestBase { - - protected String resultPath; - - public RemoveDuplicatesITCase(){ - } - - static final String[] EXPECTED_RESULT = new String[] { - "k1", "k5", "k2", "k3"}; - - @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 { - - List strings = Arrays.asList("k1", "k5", "k5", "k2", "k1", "k2", "k3"); - - Pipeline p = FlinkTestPipeline.createForBatch(); - - PCollection input = - p.apply(Create.of(strings)) - .setCoder(StringUtf8Coder.of()); - - PCollection output = - input.apply(RemoveDuplicates.create()); - - output.apply(TextIO.Write.to(resultPath)); - p.run(); - } -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java deleted file mode 100644 index 2c7c65e8af3d..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.values.PCollectionView; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.io.Serializable; - -public class SideInputITCase extends JavaProgramTestBase implements Serializable { - - private static final String expected = "Hello!"; - - protected String resultPath; - - @Override - protected void testProgram() throws Exception { - - - Pipeline p = FlinkTestPipeline.createForBatch(); - - - 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.run(); - } - - @Override - protected void preSubmit() throws Exception { - resultPath = getTempDirPath("result"); - } - - @Override - protected void postSubmit() throws Exception { - compareResultsByLinesInMemory(expected, resultPath); - } -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java deleted file mode 100644 index 547f3c3a4660..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.examples.complete.TfIdf; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.StringDelegateCoder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.Keys; -import org.apache.beam.sdk.transforms.RemoveDuplicates; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.net.URI; - - -public class TfIdfITCase extends JavaProgramTestBase { - - protected String resultPath; - - public TfIdfITCase(){ - } - - 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 postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); - } - - @Override - protected void testProgram() throws Exception { - - Pipeline pipeline = FlinkTestPipeline.createForBatch(); - - 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 words = wordToUriAndTfIdf - .apply(Keys.create()) - .apply(RemoveDuplicates.create()); - - words.apply(TextIO.Write.to(resultPath)); - - pipeline.run(); - } -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java deleted file mode 100644 index 3254e7885db8..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.examples.WordCount; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - -import java.util.Arrays; -import java.util.List; - - -public class WordCountITCase extends JavaProgramTestBase { - - protected String resultPath; - - public WordCountITCase(){ - } - - 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 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 postSubmit() throws Exception { - compareResultsByLinesInMemory(Joiner.on('\n').join(COUNTS_ARRAY), resultPath); - } - - @Override - protected void testProgram() throws Exception { - - Pipeline p = FlinkTestPipeline.createForBatch(); - - 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)); - - p.run(); - } -} - diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java deleted file mode 100644 index 6570e7df5508..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGroupByKey; -import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - - -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); - } - } - - -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java deleted file mode 100644 index 60dc74af90b6..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGroupByKey; -import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.base.Joiner; - -import org.apache.flink.test.util.JavaProgramTestBase; - - -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); - } - } - -} diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java index f3ceba76077a..a488768467ea 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java @@ -43,6 +43,7 @@ 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.StreamingMultipleProgramsTestBase; import org.apache.flink.streaming.util.TestHarnessUtil; import org.joda.time.Duration; import org.joda.time.Instant; @@ -52,7 +53,7 @@ import java.util.Comparator; import java.util.concurrent.ConcurrentLinkedQueue; -public class GroupAlsoByWindowTest { +public class GroupAlsoByWindowTest extends StreamingMultipleProgramsTestBase { private final Combine.CombineFn combiner = new Sum.SumIntegerFn(); diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java deleted file mode 100644 index e6b7f64f69a1..000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.flink.util; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.BigQueryIO; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.Validation; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.transforms.join.CoGroupByKey; -import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TupleTag; - -import com.google.api.services.bigquery.model.TableRow; - -/** - * Copied from {@link org.apache.beam.examples.JoinExamples} because the code - * is private there. - */ -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(); - } - -} diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 55e7ed93145b..a1d48ce2e4d2 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -35,7 +35,6 @@ jar - 0.1.0-incubating-SNAPSHOT ${maven.build.timestamp} yyyy-MM-dd HH:mm @@ -108,43 +107,12 @@ org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - @@ -220,7 +188,6 @@ org.apache.maven.plugins maven-shade-plugin - 2.4.1 @@ -318,194 +285,110 @@ org.apache.beam java-sdk-all - ${beam.version} com.google.api-client google-api-client - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - com.google.oauth-client google-oauth-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client-jackson2 - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client-protobuf - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - runtime com.google.apis google-api-services-dataflow - ${dataflow.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-clouddebugger - ${clouddebugger.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-bigquery - ${bigquery.version} - - - - com.google.guava - guava-jdk5 - - com.google.cloud.bigdataoss util - 1.4.3 com.google.guava guava - - ${guava.version} com.google.guava guava-testlib - ${guava.version} test joda-time joda-time - ${joda.version} com.google.protobuf protobuf-java - ${protobuf.version} com.google.code.findbugs jsr305 - ${jsr305.version} com.fasterxml.jackson.core jackson-core - ${jackson.version} com.fasterxml.jackson.core jackson-annotations - ${jackson.version} com.fasterxml.jackson.core jackson-databind - ${jackson.version} org.slf4j slf4j-api - ${slf4j.version} com.google.auto.service auto-service - 1.0-rc2 true com.google.auto.value auto-value - 1.1 provided @@ -513,35 +396,30 @@ org.hamcrest hamcrest-all - ${hamcrest.version} provided junit junit - ${junit.version} provided org.slf4j slf4j-jdk14 - ${slf4j.version} test org.mockito mockito-all - 1.10.19 test org.apache.beam java-sdk-all - ${beam.version} test-jar test @@ -549,7 +427,6 @@ com.google.cloud.dataflow google-cloud-dataflow-java-proto-library-all - 0.5.160304 test diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index 69565ac6573e..0f42148a0a4c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -41,7 +41,6 @@ import com.google.api.services.dataflow.model.JobMetrics; import com.google.api.services.dataflow.model.MetricUpdate; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Throwables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -356,7 +355,7 @@ private boolean nextBackOff(Sleeper sleeper, BackOff backoff) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } - throw Throwables.propagate(e); + throw new RuntimeException(e); } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java index 3ab91f5511df..2b53a655a0f9 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java @@ -17,18 +17,20 @@ */ package org.apache.beam.runners.dataflow.testing; +import static org.hamcrest.MatcherAssert.assertThat; + import org.apache.beam.runners.dataflow.DataflowJobExecutionException; import org.apache.beam.runners.dataflow.DataflowPipelineJob; import org.apache.beam.runners.dataflow.DataflowPipelineRunner; import org.apache.beam.runners.dataflow.util.MonitoringUtil; import org.apache.beam.runners.dataflow.util.MonitoringUtil.JobMessagesHandler; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineResult.State; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; @@ -39,16 +41,13 @@ import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Throwables; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.math.BigDecimal; import java.util.List; -import java.util.Map; import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -62,8 +61,6 @@ public class TestDataflowPipelineRunner extends PipelineRunner { private static final String TENTATIVE_COUNTER = "tentative"; private static final Logger LOG = LoggerFactory.getLogger(TestDataflowPipelineRunner.class); - private static final Map EXECUTION_RESULTS = - new ConcurrentHashMap(); private final TestDataflowPipelineOptions options; private final DataflowPipelineRunner runner; @@ -87,10 +84,6 @@ public static TestDataflowPipelineRunner fromOptions( return new TestDataflowPipelineRunner(dataflowOptions); } - public static PipelineResult getPipelineResultByJobName(String jobName) { - return EXECUTION_RESULTS.get(jobName); - } - @Override public DataflowPipelineJob run(Pipeline pipeline) { return run(pipeline, runner); @@ -98,6 +91,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { DataflowPipelineJob run(Pipeline pipeline, DataflowPipelineRunner runner) { + TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class); final DataflowPipelineJob job; try { job = runner.run(pipeline); @@ -108,6 +102,8 @@ DataflowPipelineJob run(Pipeline pipeline, DataflowPipelineRunner runner) { LOG.info("Running Dataflow job {} with {} expected assertions.", job.getJobId(), expectedNumberOfAssertions); + assertThat(job, testPipelineOptions.getOnCreateMatcher()); + CancelWorkflowOnError messageHandler = new CancelWorkflowOnError( job, new MonitoringUtil.PrintHandler(options.getJobMessageOutput())); @@ -151,6 +147,8 @@ public Optional call() throws Exception { throw new AssertionError(messageHandler.getErrorMessage() == null ? "The dataflow did not return a failure reason." : messageHandler.getErrorMessage()); + } else { + assertThat(job, testPipelineOptions.getOnSuccessMatcher()); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -161,7 +159,6 @@ public Optional call() throws Exception { } catch (IOException e) { throw new RuntimeException(e); } - EXECUTION_RESULTS.put(options.getJobName(), job); return job; } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java index a45284c9afa9..fbaf116d98cb 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java @@ -19,6 +19,7 @@ import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -36,10 +37,13 @@ import org.apache.beam.runners.dataflow.util.MonitoringUtil.JobMessagesHandler; import org.apache.beam.runners.dataflow.util.TimeUtil; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineResult.State; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.SerializableMatcher; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.NoopPathValidator; @@ -61,6 +65,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; @@ -74,6 +80,7 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import java.io.IOException; import java.math.BigDecimal; import java.util.Arrays; import java.util.concurrent.TimeUnit; @@ -378,4 +385,217 @@ public State answer(InvocationOnMock invocation) { // instead of inside the try-catch block. fail("AssertionError expected"); } + + @Test + public void testBatchOnCreateMatcher() throws Exception { + Pipeline p = TestPipeline.create(options); + PCollection pc = p.apply(Create.of(1, 2, 3)); + PAssert.that(pc).containsInAnyOrder(1, 2, 3); + + final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); + when(mockJob.getDataflowClient()).thenReturn(service); + when(mockJob.getState()).thenReturn(State.DONE); + when(mockJob.getProjectId()).thenReturn("test-project"); + when(mockJob.getJobId()).thenReturn("test-job"); + + DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class); + when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); + + TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner(); + p.getOptions().as(TestPipelineOptions.class) + .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0)); + + when(request.execute()).thenReturn( + generateMockMetricResponse(true /* success */, true /* tentative */)); + runner.run(p, mockRunner); + } + + @Test + public void testStreamingOnCreateMatcher() throws Exception { + options.setStreaming(true); + Pipeline p = TestPipeline.create(options); + PCollection pc = p.apply(Create.of(1, 2, 3)); + PAssert.that(pc).containsInAnyOrder(1, 2, 3); + + final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); + when(mockJob.getDataflowClient()).thenReturn(service); + when(mockJob.getState()).thenReturn(State.DONE); + when(mockJob.getProjectId()).thenReturn("test-project"); + when(mockJob.getJobId()).thenReturn("test-job"); + + DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class); + when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); + + TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner(); + p.getOptions().as(TestPipelineOptions.class) + .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0)); + + when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class))) + .thenReturn(State.DONE); + + when(request.execute()).thenReturn( + generateMockMetricResponse(true /* success */, true /* tentative */)); + runner.run(p, mockRunner); + } + + @Test + public void testBatchOnSuccessMatcherWhenPipelineSucceeds() throws Exception { + Pipeline p = TestPipeline.create(options); + PCollection pc = p.apply(Create.of(1, 2, 3)); + PAssert.that(pc).containsInAnyOrder(1, 2, 3); + + final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); + when(mockJob.getDataflowClient()).thenReturn(service); + when(mockJob.getState()).thenReturn(State.DONE); + when(mockJob.getProjectId()).thenReturn("test-project"); + when(mockJob.getJobId()).thenReturn("test-job"); + + DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class); + when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); + + TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner(); + p.getOptions().as(TestPipelineOptions.class) + .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1)); + + when(request.execute()).thenReturn( + generateMockMetricResponse(true /* success */, true /* tentative */)); + runner.run(p, mockRunner); + } + + @Test + public void testStreamingOnSuccessMatcherWhenPipelineSucceeds() throws Exception { + options.setStreaming(true); + Pipeline p = TestPipeline.create(options); + PCollection pc = p.apply(Create.of(1, 2, 3)); + PAssert.that(pc).containsInAnyOrder(1, 2, 3); + + final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); + when(mockJob.getDataflowClient()).thenReturn(service); + when(mockJob.getState()).thenReturn(State.DONE); + when(mockJob.getProjectId()).thenReturn("test-project"); + when(mockJob.getJobId()).thenReturn("test-job"); + + DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class); + when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); + + TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner(); + p.getOptions().as(TestPipelineOptions.class) + .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1)); + + when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class))) + .thenReturn(State.DONE); + + when(request.execute()).thenReturn( + generateMockMetricResponse(true /* success */, true /* tentative */)); + runner.run(p, mockRunner); + } + + @Test + public void testBatchOnSuccessMatcherWhenPipelineFails() throws Exception { + Pipeline p = TestPipeline.create(options); + PCollection pc = p.apply(Create.of(1, 2, 3)); + PAssert.that(pc).containsInAnyOrder(1, 2, 3); + + final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); + when(mockJob.getDataflowClient()).thenReturn(service); + when(mockJob.getState()).thenReturn(State.FAILED); + when(mockJob.getProjectId()).thenReturn("test-project"); + when(mockJob.getJobId()).thenReturn("test-job"); + + DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class); + when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); + + TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner(); + p.getOptions().as(TestPipelineOptions.class) + .setOnSuccessMatcher(new TestFailureMatcher()); + + when(request.execute()).thenReturn( + generateMockMetricResponse(false /* success */, true /* tentative */)); + try { + runner.run(p, mockRunner); + } catch (AssertionError expected) { + verify(mockJob, Mockito.times(1)).waitToFinish(any(Long.class), any(TimeUnit.class), + any(JobMessagesHandler.class)); + return; + } + fail("Expected an exception on pipeline failure."); + } + + @Test + public void testStreamingOnSuccessMatcherWhenPipelineFails() throws Exception { + options.setStreaming(true); + Pipeline p = TestPipeline.create(options); + PCollection pc = p.apply(Create.of(1, 2, 3)); + PAssert.that(pc).containsInAnyOrder(1, 2, 3); + + final DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class); + when(mockJob.getDataflowClient()).thenReturn(service); + when(mockJob.getState()).thenReturn(State.FAILED); + when(mockJob.getProjectId()).thenReturn("test-project"); + when(mockJob.getJobId()).thenReturn("test-job"); + + DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class); + when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); + + TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner(); + p.getOptions().as(TestPipelineOptions.class) + .setOnSuccessMatcher(new TestFailureMatcher()); + + when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class))) + .thenReturn(State.FAILED); + + when(request.execute()).thenReturn( + generateMockMetricResponse(false /* success */, true /* tentative */)); + try { + runner.run(p, mockRunner); + } catch (AssertionError expected) { + verify(mockJob, Mockito.times(1)).waitToFinish(any(Long.class), any(TimeUnit.class), + any(JobMessagesHandler.class)); + return; + } + fail("Expected an exception on pipeline failure."); + } + + static class TestSuccessMatcher extends BaseMatcher implements + SerializableMatcher { + private final DataflowPipelineJob mockJob; + private final int called; + + public TestSuccessMatcher(DataflowPipelineJob job, int times) { + this.mockJob = job; + this.called = times; + } + + @Override + public boolean matches(Object o) { + if (!(o instanceof PipelineResult)) { + fail(String.format("Expected PipelineResult but received %s", o)); + } + try { + verify(mockJob, Mockito.times(called)).waitToFinish(any(Long.class), any(TimeUnit.class), + any(JobMessagesHandler.class)); + } catch (IOException | InterruptedException e) { + throw new AssertionError(e); + } + assertSame(mockJob, o); + return true; + } + + @Override + public void describeTo(Description description) { + } + } + + static class TestFailureMatcher extends BaseMatcher implements + SerializableMatcher { + @Override + public boolean matches(Object o) { + fail("OnSuccessMatcher should not be called on pipeline failure."); + return false; + } + + @Override + public void describeTo(Description description) { + } + } } diff --git a/runners/pom.xml b/runners/pom.xml index 74812e8d0cc7..d2d68dfca132 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -35,6 +35,7 @@ Apache Beam :: Runners + core-java direct-java flink spark diff --git a/runners/spark/build-resources/checkstyle.xml b/runners/spark/build-resources/checkstyle.xml deleted file mode 100644 index 033d8ae476de..000000000000 --- a/runners/spark/build-resources/checkstyle.xml +++ /dev/null @@ -1,225 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index eb0122994630..e6732464734a 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -25,6 +25,7 @@ org.apache.beam runners-parent 0.1.0-incubating-SNAPSHOT + ../pom.xml spark-runner @@ -35,9 +36,7 @@ UTF-8 UTF-8 - 1.7 1.6.1 - 0.1.0-incubating-SNAPSHOT @@ -68,18 +67,15 @@ com.google.guava guava - ${guava.version} com.google.auto.service auto-service - 1.0-rc2 true org.apache.beam java-sdk-all - ${beam.version} @@ -91,7 +87,6 @@ org.apache.beam java-examples-all - ${beam.version} @@ -118,13 +113,11 @@ junit junit - 4.12 test org.hamcrest hamcrest-all - 1.3 test @@ -132,155 +125,16 @@ - - org.apache.maven.plugins - maven-compiler-plugin - 3.5 - - ${java.version} - ${java.version} - - -Xlint:all,-serial - - - org.apache.maven.plugins maven-surefire-plugin + 2.19.1 1 false - - org.apache.maven.plugins - maven-checkstyle-plugin - - build-resources/checkstyle.xml - false - - - - org.apache.maven.plugins - maven-source-plugin - 2.4 - - - attach-sources - - jar-no-fork - - - - - - org.apache.maven.plugins - maven-javadoc-plugin - 2.10.3 - - ${java.version} - - - - attach-javadocs - - jar - - - - - - org.apache.maven.plugins - maven-clean-plugin - 3.0.0 - - - org.apache.maven.plugins - maven-deploy-plugin - 2.8.2 - - - org.apache.maven.plugins - maven-install-plugin - 2.5.2 - - true - - - - org.apache.maven.plugins - maven-resources-plugin - 2.7 - - - org.apache.maven.plugins - maven-jar-plugin - 2.6 - - - org.apache.maven.plugins - maven-site-plugin - 3.4 - - - org.apache.maven.plugins - maven-release-plugin - 2.5.3 - - - org.apache.maven.scm - maven-scm-provider-gitexe - 1.9.2 - - - - - org.apache.maven.plugins - maven-gpg-plugin - 1.6 - - - sign-artifacts - verify - - sign - - - - - - org.apache.rat - apache-rat-plugin - 0.11 - - - .travis.yml - **/*.conf - **/*.iml - **/*.md - **/*.txt - **/.project - **/.checkstyle - **/.classpath - **/.settings/ - **/gen/** - **/resources/** - **/target/** - **/dependency-reduced-pom.xml - false - - - - - verify - verify - - check - - - - org.codehaus.mojo exec-maven-plugin @@ -321,7 +175,6 @@ org.apache.maven.plugins maven-shade-plugin - 2.4.3 package @@ -352,12 +205,18 @@ org.apache.maven.plugins - maven-checkstyle-plugin + maven-compiler-plugin - org.apache.rat - apache-rat-plugin + org.apache.maven.plugins + maven-jar-plugin + + org.apache.maven.plugins maven-source-plugin diff --git a/sdks/java/src.xml b/sdks/java/build-tools/src/main/assembly/src.xml similarity index 100% rename from sdks/java/src.xml rename to sdks/java/build-tools/src/main/assembly/src.xml diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml index 31717ffda967..2a4f8327fec2 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml @@ -59,6 +59,12 @@ page at http://checkstyle.sourceforge.net/config.html --> + + + + + + diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 03d224d71b60..d1adf55883db 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - @@ -49,6 +49,75 @@ + + + + org.apache.maven.plugins + maven-javadoc-plugin + + Apache Beam SDK ${project.version} API + Apache Beam SDK for Java, version ${project.version} + ../javadoc/overview.html + + org.apache.beam.sdk + -exclude + org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util:org.apache.beam.sdk.runners.inprocess + ${dataflow.javadoc_opts} + false + true + ]]> + + + + https://developers.google.com/api-client-library/java/google-api-java-client/reference/1.20.0/ + ${basedir}/../javadoc/apiclient-docs + + + http://avro.apache.org/docs/1.7.7/api/java/ + ${basedir}/../javadoc/avro-docs + + + https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/ + ${basedir}/../javadoc/bq-docs + + + https://cloud.google.com/datastore/docs/apis/javadoc/ + ${basedir}/../javadoc/datastore-docs + + + http://docs.guava-libraries.googlecode.com/git-history/release19/javadoc/ + ${basedir}/../javadoc/guava-docs + + + http://hamcrest.org/JavaHamcrest/javadoc/1.3/ + ${basedir}/../javadoc/hamcrest-docs + + + http://fasterxml.github.io/jackson-annotations/javadoc/2.7/ + ${basedir}/../javadoc/jackson-annotations-docs + + + http://fasterxml.github.io/jackson-databind/javadoc/2.7/ + ${basedir}/../javadoc/jackson-databind-docs + + + http://www.joda.org/joda-time/apidocs + ${basedir}/../javadoc/joda-docs + + + http://junit.sourceforge.net/javadoc/ + ${basedir}/../javadoc/junit-docs + + + https://developers.google.com/api-client-library/java/google-oauth-java-client/reference/1.20.0/ + ${basedir}/../javadoc/oauth-docs + + + + + + + org.apache.maven.plugins @@ -60,7 +129,9 @@ maven-dependency-plugin - analyze-only + + analyze-only + true @@ -68,8 +139,6 @@ - org.apache.maven.plugins maven-checkstyle-plugin @@ -82,120 +151,22 @@ org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - org.apache.maven.plugins maven-javadoc-plugin - - Google Cloud Dataflow SDK ${project.version} API - Google Cloud Dataflow SDK for Java, version ${project.version} - ../javadoc/overview.html - - org.apache.beam.sdk - -exclude org.apache.beam.sdk.runners.worker:org.apache.beam.sdk.runners.dataflow:org.apache.beam.sdk.util:org.apache.beam.sdk.runners.inprocess ${dataflow.javadoc_opts} - false - true - ]]> - - - - https://developers.google.com/api-client-library/java/google-api-java-client/reference/1.20.0/ - ${basedir}/../javadoc/apiclient-docs - - - http://avro.apache.org/docs/1.7.7/api/java/ - ${basedir}/../javadoc/avro-docs - - - https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/ - ${basedir}/../javadoc/bq-docs - - - https://cloud.google.com/datastore/docs/apis/javadoc/ - ${basedir}/../javadoc/datastore-docs - - - http://docs.guava-libraries.googlecode.com/git-history/release19/javadoc/ - ${basedir}/../javadoc/guava-docs - - - http://hamcrest.org/JavaHamcrest/javadoc/1.3/ - ${basedir}/../javadoc/hamcrest-docs - - - http://fasterxml.github.io/jackson-annotations/javadoc/2.7/ - ${basedir}/../javadoc/jackson-annotations-docs - - - http://fasterxml.github.io/jackson-databind/javadoc/2.7/ - ${basedir}/../javadoc/jackson-databind-docs - - - http://www.joda.org/joda-time/apidocs - ${basedir}/../javadoc/joda-docs - - - http://junit.sourceforge.net/javadoc/ - ${basedir}/../javadoc/junit-docs - - - https://developers.google.com/api-client-library/java/google-oauth-java-client/reference/1.20.0/ - ${basedir}/../javadoc/oauth-docs - - - - - - - jar - - package - - org.apache.maven.plugins maven-shade-plugin - 2.4.1 @@ -222,9 +193,8 @@ - + com.google.common org.apache.beam.sdk.repackaged.com.google.common @@ -237,8 +207,8 @@ - + bundle-rest-without-repackaging package @@ -278,7 +248,6 @@ org.apache.avro avro-maven-plugin - ${avro.version} schemas @@ -294,14 +263,12 @@ - + org.codehaus.mojo build-helper-maven-plugin - 1.9.1 add-test-source @@ -324,7 +291,6 @@ io.grpc grpc-all - 0.12.0 @@ -332,8 +298,8 @@ google-auth-library-oauth2-http 0.3.1 - + com.google.guava guava-jdk5 @@ -352,8 +318,8 @@ grpc-pubsub-v1 ${pubsubgrpc.version} - + com.google.guava guava-jdk5 @@ -364,251 +330,116 @@ com.google.api-client google-api-client - ${google-clients.version} - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-bigquery - ${bigquery.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-pubsub - ${pubsub.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-storage - ${storage.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - - + com.google.http-client google-http-client-jackson - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - - - org.codehaus.jackson - jackson-core-asl - - runtime com.google.http-client google-http-client-jackson2 - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.http-client google-http-client-protobuf - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - runtime com.google.oauth-client google-oauth-client-java6 - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.oauth-client google-oauth-client - ${google-clients.version} - - - - com.google.guava - guava-jdk5 - - com.google.apis google-api-services-datastore-protobuf - ${datastore.version} - - - - com.google.guava - guava-jdk5 - - - - com.google.http-client - google-http-client - - - com.google.api-client - google-api-client - - - com.google.oauth-client - google-oauth-client - - - com.google.http-client - google-http-client-jackson - - - com.google.http-client - google-http-client-protobuf - - com.google.cloud.bigdataoss gcsio - 1.4.3 com.google.cloud.bigdataoss util - 1.4.3 com.google.guava guava - - ${guava.version} com.google.guava guava-testlib - ${guava.version} test com.google.protobuf protobuf-java - ${protobuf.version} com.google.code.findbugs jsr305 - ${jsr305.version} com.fasterxml.jackson.core jackson-core - ${jackson.version} com.fasterxml.jackson.core jackson-annotations - ${jackson.version} com.fasterxml.jackson.core jackson-databind - ${jackson.version} org.slf4j slf4j-api - ${slf4j.version} org.apache.avro avro - ${avro.version} @@ -626,11 +457,10 @@ joda-time joda-time - ${joda.version} - + org.tukaani xz @@ -675,14 +502,12 @@ com.google.auto.service auto-service - 1.0-rc2 true com.google.auto.value auto-value - 1.1 provided @@ -690,35 +515,30 @@ org.hamcrest hamcrest-all - ${hamcrest.version} provided junit junit - ${junit.version} provided org.slf4j slf4j-jdk14 - ${slf4j.version} test org.mockito mockito-all - 1.10.19 test com.google.cloud.dataflow google-cloud-dataflow-java-proto-library-all - 0.5.160304 test diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java index 4a0bdacb6970..77852989dd97 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java @@ -47,7 +47,7 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.BigQueryServices; -import org.apache.beam.sdk.util.BigQueryServices.LoadService; +import org.apache.beam.sdk.util.BigQueryServices.JobService; import org.apache.beam.sdk.util.BigQueryServicesImpl; import org.apache.beam.sdk.util.BigQueryTableInserter; import org.apache.beam.sdk.util.BigQueryTableRowIterator; @@ -68,7 +68,9 @@ import com.google.api.client.json.JsonFactory; import com.google.api.services.bigquery.Bigquery; +import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobStatus; import com.google.api.services.bigquery.model.QueryRequest; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; @@ -1168,6 +1170,10 @@ private static class BigQueryWriteOperation extends FileBasedWriteOperation writerResults, PipelineOptions options } if (!tempFiles.isEmpty()) { load( - bigQuerySink.bqServices.getLoadService(bqOptions), + bigQuerySink.bqServices.getJobService(bqOptions), bigQuerySink.jobIdToken, fromJsonString(bigQuerySink.jsonTable, TableReference.class), tempFiles, @@ -1215,7 +1221,7 @@ public void finalize(Iterable writerResults, PipelineOptions options *

If a load job failed, it will try another load job with a different job id. */ private void load( - LoadService loadService, + JobService jobService, String jobIdPrefix, TableReference ref, List gcsUris, @@ -1238,8 +1244,9 @@ private void load( LOG.info("Previous load jobs failed, retrying."); } LOG.info("Starting BigQuery load job: {}", jobId); - loadService.startLoadJob(jobId, loadConfig); - BigQueryServices.Status jobStatus = loadService.pollJobStatus(projectId, jobId); + jobService.startLoadJob(jobId, loadConfig); + Status jobStatus = parseStatus( + jobService.pollJob(projectId, jobId, MAX_JOB_STATUS_POLL_RETRIES)); switch (jobStatus) { case SUCCEEDED: return; @@ -1669,6 +1676,29 @@ public PDone apply(PCollection input) { } } + /** + * Status of a BigQuery job or request. + */ + enum Status { + SUCCEEDED, + FAILED, + UNKNOWN, + } + + private static Status parseStatus(@Nullable Job job) { + if (job == null) { + return Status.UNKNOWN; + } + JobStatus status = job.getStatus(); + if (status.getErrorResult() != null) { + return Status.FAILED; + } else if (status.getErrors() != null && !status.getErrors().isEmpty()) { + return Status.FAILED; + } else { + return Status.SUCCEEDED; + } + } + private static String toJsonString(Object item) { if (item == null) { return null; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java deleted file mode 100644 index f92b480de1d8..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubClient.java +++ /dev/null @@ -1,323 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.sdk.io; - -import static com.google.common.base.Preconditions.checkState; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; - -/** - * A helper interface for talking to Pubsub via an underlying transport. - */ -public interface PubsubClient extends AutoCloseable { - /** - * Path representing a cloud project id. - */ - class ProjectPath implements Serializable { - private final String path; - - public ProjectPath(String path) { - this.path = path; - } - - public String getPath() { - return path; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - ProjectPath that = (ProjectPath) o; - - return path.equals(that.path); - - } - - @Override - public int hashCode() { - return path.hashCode(); - } - - @Override - public String toString() { - return path; - } - - public static ProjectPath fromId(String projectId) { - return new ProjectPath(String.format("projects/%s", projectId)); - } - } - - /** - * Path representing a Pubsub subscription. - */ - class SubscriptionPath implements Serializable { - private final String path; - - public SubscriptionPath(String path) { - this.path = path; - } - - public String getPath() { - return path; - } - - public String getV1Beta1Path() { - String[] splits = path.split("/"); - checkState(splits.length == 4); - return String.format("/subscriptions/%s/%s", splits[1], splits[3]); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - SubscriptionPath that = (SubscriptionPath) o; - return path.equals(that.path); - } - - @Override - public int hashCode() { - return path.hashCode(); - } - - @Override - public String toString() { - return path; - } - - public static SubscriptionPath fromName(String projectId, String subscriptionName) { - return new SubscriptionPath(String.format("projects/%s/subscriptions/%s", - projectId, subscriptionName)); - } - } - - /** - * Path representing a Pubsub topic. - */ - class TopicPath implements Serializable { - private final String path; - - public TopicPath(String path) { - this.path = path; - } - - public String getPath() { - return path; - } - - public String getV1Beta1Path() { - String[] splits = path.split("/"); - checkState(splits.length == 4); - return String.format("/topics/%s/%s", splits[1], splits[3]); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TopicPath topicPath = (TopicPath) o; - return path.equals(topicPath.path); - } - - @Override - public int hashCode() { - return path.hashCode(); - } - - @Override - public String toString() { - return path; - } - - public static TopicPath fromName(String projectId, String topicName) { - return new TopicPath(String.format("projects/%s/topics/%s", projectId, topicName)); - } - } - - /** - * A message to be sent to Pubsub. - */ - class OutgoingMessage { - /** - * Underlying (encoded) element. - */ - public final byte[] elementBytes; - - /** - * Timestamp for element (ms since epoch). - */ - public final long timestampMsSinceEpoch; - - public OutgoingMessage(byte[] elementBytes, long timestampMsSinceEpoch) { - this.elementBytes = elementBytes; - this.timestampMsSinceEpoch = timestampMsSinceEpoch; - } - } - - /** - * A message received from Pubsub. - */ - class IncomingMessage { - /** - * Underlying (encoded) element. - */ - public final byte[] elementBytes; - - /** - * Timestamp for element (ms since epoch). Either Pubsub's processing time, - * or the custom timestamp associated with the message. - */ - public final long timestampMsSinceEpoch; - - /** - * Timestamp (in system time) at which we requested the message (ms since epoch). - */ - public final long requestTimeMsSinceEpoch; - - /** - * Id to pass back to Pubsub to acknowledge receipt of this message. - */ - public final String ackId; - - /** - * Id to pass to the runner to distinguish this message from all others. - */ - public final byte[] recordId; - - public IncomingMessage( - byte[] elementBytes, - long timestampMsSinceEpoch, - long requestTimeMsSinceEpoch, - String ackId, - byte[] recordId) { - this.elementBytes = elementBytes; - this.timestampMsSinceEpoch = timestampMsSinceEpoch; - this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch; - this.ackId = ackId; - this.recordId = recordId; - } - } - - /** - * Gracefully close the underlying transport. - */ - @Override - void close(); - - - /** - * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages - * published. - * - * @throws IOException - */ - int publish(TopicPath topic, Iterable outgoingMessages) throws IOException; - - /** - * Request the next batch of up to {@code batchSize} messages from {@code subscription}. - * Return the received messages, or empty collection if none were available. Does not - * wait for messages to arrive. Returned messages will record heir request time - * as {@code requestTimeMsSinceEpoch}. - * - * @throws IOException - */ - Collection pull( - long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize) - throws IOException; - - /** - * Acknowldege messages from {@code subscription} with {@code ackIds}. - * - * @throws IOException - */ - void acknowledge(SubscriptionPath subscription, Iterable ackIds) throws IOException; - - /** - * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to - * be {@code deadlineSeconds} from now. - * - * @throws IOException - */ - void modifyAckDeadline( - SubscriptionPath subscription, Iterable ackIds, - int deadlineSeconds) - throws IOException; - - /** - * Create {@code topic}. - * - * @throws IOException - */ - void createTopic(TopicPath topic) throws IOException; - - /* - * Delete {@code topic}. - * - * @throws IOException - */ - void deleteTopic(TopicPath topic) throws IOException; - - /** - * Return a list of topics for {@code project}. - * - * @throws IOException - */ - Collection listTopics(ProjectPath project) throws IOException; - - /** - * Create {@code subscription} to {@code topic}. - * - * @throws IOException - */ - void createSubscription( - TopicPath topic, SubscriptionPath subscription, - int ackDeadlineSeconds) throws IOException; - - /** - * Delete {@code subscription}. - * - * @throws IOException - */ - void deleteSubscription(SubscriptionPath subscription) throws IOException; - - /** - * Return a list of subscriptions for {@code topic} in {@code project}. - * - * @throws IOException - */ - Collection listSubscriptions(ProjectPath project, TopicPath topic) - throws IOException; -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java index 66d1d4335505..fa867c24bd22 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java @@ -17,8 +17,7 @@ */ package org.apache.beam.sdk.io; -import static com.google.common.base.MoreObjects.firstNonNull; -import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -33,26 +32,18 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.Transport; +import org.apache.beam.sdk.util.PubsubApiaryClient; +import org.apache.beam.sdk.util.PubsubClient; +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; -import com.google.api.client.util.Clock; -import com.google.api.client.util.DateTime; -import com.google.api.services.pubsub.Pubsub; -import com.google.api.services.pubsub.model.AcknowledgeRequest; -import com.google.api.services.pubsub.model.PublishRequest; -import com.google.api.services.pubsub.model.PubsubMessage; -import com.google.api.services.pubsub.model.PullRequest; -import com.google.api.services.pubsub.model.PullResponse; -import com.google.api.services.pubsub.model.ReceivedMessage; -import com.google.api.services.pubsub.model.Subscription; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; +import com.google.common.base.Strings; import org.joda.time.Duration; import org.joda.time.Instant; @@ -62,13 +53,10 @@ import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Random; import java.util.regex.Matcher; import java.util.regex.Pattern; - import javax.annotation.Nullable; /** @@ -83,6 +71,9 @@ public class PubsubIO { private static final Logger LOG = LoggerFactory.getLogger(PubsubIO.class); + /** Factory for creating pubsub client to manage transport. */ + private static final PubsubClient.PubsubClientFactory FACTORY = PubsubApiaryClient.FACTORY; + /** The default {@link Coder} used to translate to/from Cloud Pub/Sub messages. */ public static final Coder DEFAULT_PUBSUB_CODER = StringUtf8Coder.of(); @@ -143,48 +134,6 @@ private static void validatePubsubName(String name) { } } - /** - * Returns the {@link Instant} that corresponds to the timestamp in the supplied - * {@link PubsubMessage} under the specified {@code ink label}. See - * {@link PubsubIO.Read#timestampLabel(String)} for details about how these messages are - * parsed. - * - *

The {@link Clock} parameter is used to virtualize time for testing. - * - * @throws IllegalArgumentException if the timestamp label is provided, but there is no - * corresponding attribute in the message or the value provided is not a valid timestamp - * string. - * @see PubsubIO.Read#timestampLabel(String) - */ - @VisibleForTesting - protected static Instant assignMessageTimestamp( - PubsubMessage message, @Nullable String label, Clock clock) { - if (label == null) { - return new Instant(clock.currentTimeMillis()); - } - - // Extract message attributes, defaulting to empty map if null. - Map attributes = firstNonNull( - message.getAttributes(), ImmutableMap.of()); - - String timestampStr = attributes.get(label); - checkArgument(timestampStr != null && !timestampStr.isEmpty(), - "PubSub message is missing a timestamp in label: %s", label); - - long millisSinceEpoch; - try { - // Try parsing as milliseconds since epoch. Note there is no way to parse a string in - // RFC 3339 format here. - // Expected IllegalArgumentException if parsing fails; we use that to fall back to RFC 3339. - millisSinceEpoch = Long.parseLong(timestampStr); - } catch (IllegalArgumentException e) { - // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an IllegalArgumentException - // if parsing fails, and the caller should handle. - millisSinceEpoch = DateTime.parseRfc3339(timestampStr).getValue(); - } - return new Instant(millisSinceEpoch); - } - /** * Class representing a Cloud Pub/Sub Subscription. */ @@ -680,8 +629,8 @@ public PCollection apply(PInput input) { if (boundedOutput) { return input.getPipeline().begin() - .apply(Create.of((Void) null)).setCoder(VoidCoder.of()) - .apply(ParDo.of(new PubsubReader())).setCoder(coder); + .apply(Create.of((Void) null)).setCoder(VoidCoder.of()) + .apply(ParDo.of(new PubsubBoundedReader())).setCoder(coder); } else { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) @@ -741,86 +690,94 @@ public Duration getMaxReadTime() { return maxReadTime; } - private class PubsubReader extends DoFn { + /** + * Default reader when Pubsub subscription has some form of upper bound. + *

TODO: Consider replacing with BoundedReadFromUnboundedSource on top of upcoming + * PubsubUnboundedSource. + *

NOTE: This is not the implementation used when running on the Google Dataflow hosted + * service. + */ + private class PubsubBoundedReader extends DoFn { private static final int DEFAULT_PULL_SIZE = 100; + private static final int ACK_TIMEOUT_SEC = 60; @Override public void processElement(ProcessContext c) throws IOException { - Pubsub pubsubClient = - Transport.newPubsubClient(c.getPipelineOptions().as(PubsubOptions.class)) - .build(); - - String subscription; - if (getSubscription() == null) { - String topic = getTopic().asPath(); - String[] split = topic.split("/"); - subscription = - "projects/" + split[1] + "/subscriptions/" + split[3] + "_dataflow_" - + new Random().nextLong(); - Subscription subInfo = new Subscription().setAckDeadlineSeconds(60).setTopic(topic); - try { - pubsubClient.projects().subscriptions().create(subscription, subInfo).execute(); - } catch (Exception e) { - throw new RuntimeException("Failed to create subscription: ", e); + try (PubsubClient pubsubClient = + FACTORY.newClient(timestampLabel, idLabel, + c.getPipelineOptions().as(PubsubOptions.class))) { + + PubsubClient.SubscriptionPath subscriptionPath; + if (getSubscription() == null) { + // Create a randomized subscription derived from the topic name. + String subscription = getTopic().topic + "_dataflow_" + new Random().nextLong(); + // The subscription will be registered under this pipeline's project if we know it. + // Otherwise we'll fall back to the topic's project. + // Note that they don't need to be the same. + String project = c.getPipelineOptions().as(PubsubOptions.class).getProject(); + if (Strings.isNullOrEmpty(project)) { + project = getTopic().project; + } + subscriptionPath = PubsubClient.subscriptionPathFromName(project, subscription); + TopicPath topicPath = + PubsubClient.topicPathFromName(getTopic().project, getTopic().topic); + try { + pubsubClient.createSubscription(topicPath, subscriptionPath, ACK_TIMEOUT_SEC); + } catch (Exception e) { + throw new RuntimeException("Failed to create subscription: ", e); + } + } else { + subscriptionPath = + PubsubClient.subscriptionPathFromName(getSubscription().project, + getSubscription().subscription); } - } else { - subscription = getSubscription().asPath(); - } - Instant endTime = (getMaxReadTime() == null) - ? new Instant(Long.MAX_VALUE) : Instant.now().plus(getMaxReadTime()); + Instant endTime = (getMaxReadTime() == null) + ? new Instant(Long.MAX_VALUE) : Instant.now().plus(getMaxReadTime()); - List messages = new ArrayList<>(); + List messages = new ArrayList<>(); - Throwable finallyBlockException = null; - try { - while ((getMaxNumRecords() == 0 || messages.size() < getMaxNumRecords()) - && Instant.now().isBefore(endTime)) { - PullRequest pullRequest = new PullRequest().setReturnImmediately(false); - if (getMaxNumRecords() > 0) { - pullRequest.setMaxMessages(getMaxNumRecords() - messages.size()); - } else { - pullRequest.setMaxMessages(DEFAULT_PULL_SIZE); - } + Throwable finallyBlockException = null; + try { + while ((getMaxNumRecords() == 0 || messages.size() < getMaxNumRecords()) + && Instant.now().isBefore(endTime)) { + int batchSize = DEFAULT_PULL_SIZE; + if (getMaxNumRecords() > 0) { + batchSize = Math.min(batchSize, getMaxNumRecords() - messages.size()); + } - PullResponse pullResponse = - pubsubClient.projects().subscriptions().pull(subscription, pullRequest).execute(); - List ackIds = new ArrayList<>(); - if (pullResponse.getReceivedMessages() != null) { - for (ReceivedMessage received : pullResponse.getReceivedMessages()) { - messages.add(received.getMessage()); - ackIds.add(received.getAckId()); + List batchMessages = + pubsubClient.pull(System.currentTimeMillis(), subscriptionPath, batchSize, + false); + List ackIds = new ArrayList<>(); + for (IncomingMessage message : batchMessages) { + messages.add(message); + ackIds.add(message.ackId); + } + if (ackIds.size() != 0) { + pubsubClient.acknowledge(subscriptionPath, ackIds); } } - - if (ackIds.size() != 0) { - AcknowledgeRequest ackRequest = new AcknowledgeRequest().setAckIds(ackIds); - pubsubClient.projects() - .subscriptions() - .acknowledge(subscription, ackRequest) - .execute(); + } catch (IOException e) { + throw new RuntimeException("Unexpected exception while reading from Pubsub: ", e); + } finally { + if (getSubscription() == null) { + try { + pubsubClient.deleteSubscription(subscriptionPath); + } catch (Exception e) { + finallyBlockException = e; + } } } - } catch (IOException e) { - throw new RuntimeException("Unexpected exception while reading from Pubsub: ", e); - } finally { - if (getTopic() != null) { - try { - pubsubClient.projects().subscriptions().delete(subscription).execute(); - } catch (IOException e) { - finallyBlockException = new RuntimeException("Failed to delete subscription: ", e); - LOG.error("Failed to delete subscription: ", e); - } + if (finallyBlockException != null) { + throw new RuntimeException("Failed to delete subscription: ", finallyBlockException); } - } - if (finallyBlockException != null) { - Throwables.propagate(finallyBlockException); - } - for (PubsubMessage message : messages) { - c.outputWithTimestamp( - CoderUtils.decodeFromByteArray(getCoder(), message.decodeData()), - assignMessageTimestamp(message, getTimestampLabel(), Clock.SYSTEM)); + for (IncomingMessage message : messages) { + c.outputWithTimestamp( + CoderUtils.decodeFromByteArray(getCoder(), message.elementBytes), + new Instant(message.timestampMsSinceEpoch)); + } } } } @@ -1027,31 +984,28 @@ public Coder getCoder() { return coder; } + /** + * Writer to Pubsub which batches messages. + *

NOTE: This is not the implementation used when running on the Google Dataflow hosted + * service. + */ private class PubsubWriter extends DoFn { private static final int MAX_PUBLISH_BATCH_SIZE = 100; - private transient List output; - private transient Pubsub pubsubClient; + private transient List output; + private transient PubsubClient pubsubClient; @Override - public void startBundle(Context c) { + public void startBundle(Context c) throws IOException { this.output = new ArrayList<>(); - this.pubsubClient = - Transport.newPubsubClient(c.getPipelineOptions().as(PubsubOptions.class)) - .build(); + this.pubsubClient = FACTORY.newClient(timestampLabel, idLabel, + c.getPipelineOptions().as(PubsubOptions.class)); } @Override public void processElement(ProcessContext c) throws IOException { - PubsubMessage message = - new PubsubMessage().encodeData(CoderUtils.encodeToByteArray(getCoder(), c.element())); - if (getTimestampLabel() != null) { - Map attributes = message.getAttributes(); - if (attributes == null) { - attributes = new HashMap<>(); - message.setAttributes(attributes); - } - attributes.put(getTimestampLabel(), String.valueOf(c.timestamp().getMillis())); - } + OutgoingMessage message = + new OutgoingMessage(CoderUtils.encodeToByteArray(getCoder(), c.element()), + c.timestamp().getMillis()); output.add(message); if (output.size() >= MAX_PUBLISH_BATCH_SIZE) { @@ -1064,13 +1018,16 @@ public void finishBundle(Context c) throws IOException { if (!output.isEmpty()) { publish(); } + output = null; + pubsubClient.close(); + pubsubClient = null; } private void publish() throws IOException { - PublishRequest publishRequest = new PublishRequest().setMessages(output); - pubsubClient.projects().topics() - .publish(getTopic().asPath(), publishRequest) - .execute(); + int n = pubsubClient.publish( + PubsubClient.topicPathFromName(getTopic().project, getTopic().topic), + output); + checkState(n == output.size()); output.clear(); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java index b6743fa93824..9cb026a9516c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java @@ -36,6 +36,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.UUID; /** @@ -52,6 +55,8 @@ */ @Experimental(Experimental.Kind.SOURCE_SINK) public class Write { + private static final Logger LOG = LoggerFactory.getLogger(Write.class); + /** * Creates a Write transform that writes to the given Sink. */ @@ -144,7 +149,9 @@ private PDone createWrite( @Override public void processElement(ProcessContext c) throws Exception { WriteOperation writeOperation = c.element(); + LOG.info("Initializing write operation {}", writeOperation); writeOperation.initialize(c.getPipelineOptions()); + LOG.debug("Done initializing write operation {}", writeOperation); // The WriteOperation is also the output of this ParDo, so it can have mutable // state. c.output(writeOperation); @@ -172,8 +179,10 @@ public void processElement(ProcessContext c) throws Exception { // Lazily initialize the Writer if (writer == null) { WriteOperation writeOperation = c.sideInput(writeOperationView); + LOG.info("Opening writer for write operation {}", writeOperation); writer = writeOperation.createWriter(c.getPipelineOptions()); writer.open(UUID.randomUUID().toString()); + LOG.debug("Done opening writer {} for operation {}", writer, writeOperationView); } try { writer.write(c.element()); @@ -211,9 +220,12 @@ public void finishBundle(Context c) throws Exception { .apply("Finalize", ParDo.of(new DoFn, Integer>() { @Override public void processElement(ProcessContext c) throws Exception { - Iterable results = c.sideInput(resultsView); WriteOperation writeOperation = c.element(); + LOG.info("Finalizing write operation {}", writeOperation); + Iterable results = c.sideInput(resultsView); + LOG.debug("Side input initialized to finalize write operation {}", writeOperation); writeOperation.finalize(results, c.getPipelineOptions()); + LOG.debug("Done finalizing write operation {}", writeOperation); } }).withSideInputs(resultsView)); return PDone.in(input.getPipeline()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java index 5fc7312d8ef3..5f2dd1129458 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.StringUtils; import org.apache.beam.sdk.util.common.ReflectHelpers; + import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; @@ -32,7 +33,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Strings; -import com.google.common.base.Throwables; import com.google.common.collect.Collections2; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableListMultimap; @@ -991,7 +991,7 @@ private static List validateClass(Class> { + @Override + public SerializableMatcher deserialize(JsonParser jsonParser, + DeserializationContext deserializationContext) + throws IOException, JsonProcessingException { + ObjectNode node = jsonParser.readValueAsTree(); + String matcher = node.get("matcher").asText(); + byte[] in = Base64.decodeBase64(matcher); + return (SerializableMatcher) SerializableUtils + .deserializeFromByteArray(in, "SerializableMatcher"); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java new file mode 100644 index 000000000000..845248604e2b --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing; + +import org.apache.beam.sdk.util.SerializableUtils; + +import com.google.api.client.util.Base64; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.SerializerProvider; + +import java.io.IOException; + +/** + * MatcherSerializer is used with Jackson to enable serialization of SerializableMatchers. + */ +class MatcherSerializer extends JsonSerializer> { + @Override + public void serialize(SerializableMatcher matcher, JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) throws IOException, JsonProcessingException { + byte[] out = SerializableUtils.serializeToByteArray(matcher); + String encodedString = Base64.encodeBase64String(out); + jsonGenerator.writeStartObject(); + jsonGenerator.writeStringField("matcher", encodedString); + jsonGenerator.writeEndObject(); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java index 9132db78726e..a465bbec32aa 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.testing; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; import org.hamcrest.Matcher; import java.io.Serializable; @@ -32,5 +34,8 @@ * * @param The type of value matched. */ -interface SerializableMatcher extends Matcher, Serializable { +@JsonSerialize(using = MatcherSerializer.class) +@JsonDeserialize(using = MatcherDeserializer.class) +public interface SerializableMatcher extends Matcher, Serializable { } + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index a51a24e81270..a4921d56be0a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -31,7 +31,11 @@ import com.google.common.base.Strings; import com.google.common.collect.Iterators; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.TreeNode; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.junit.experimental.categories.Category; @@ -39,7 +43,7 @@ import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Iterator; -import java.util.Map; +import java.util.Map.Entry; import javax.annotation.Nullable; @@ -157,16 +161,24 @@ public static PipelineOptions testingPipelineOptions() { public static String[] convertToArgs(PipelineOptions options) { try { - Map stringOpts = (Map) MAPPER.readValue( - MAPPER.writeValueAsBytes(options), Map.class).get("options"); + byte[] opts = MAPPER.writeValueAsBytes(options); + JsonParser jsonParser = MAPPER.getFactory().createParser(opts); + TreeNode node = jsonParser.readValueAsTree(); + ObjectNode optsNode = (ObjectNode) node.get("options"); ArrayList optArrayList = new ArrayList<>(); - for (Map.Entry entry : stringOpts.entrySet()) { - optArrayList.add("--" + entry.getKey() + "=" + entry.getValue()); + Iterator> entries = optsNode.fields(); + while (entries.hasNext()) { + Entry entry = entries.next(); + if (entry.getValue().isTextual()) { + optArrayList.add("--" + entry.getKey() + "=" + entry.getValue().asText()); + } else { + optArrayList.add("--" + entry.getKey() + "=" + entry.getValue()); + } } return optArrayList.toArray(new String[optArrayList.size()]); - } catch (Exception e) { - return null; + } catch (IOException e) { + throw new IllegalStateException(e); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java index 2599ae260d9f..ff553bafa85e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java @@ -17,7 +17,12 @@ */ package org.apache.beam.sdk.testing; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.PipelineOptions; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; /** * {@link TestPipelineOptions} is a set of options for test pipelines. @@ -27,4 +32,38 @@ public interface TestPipelineOptions extends PipelineOptions { String getTempRoot(); void setTempRoot(String value); + + @Default.InstanceFactory(AlwaysPassMatcherFactory.class) + SerializableMatcher getOnCreateMatcher(); + void setOnCreateMatcher(SerializableMatcher value); + + @Default.InstanceFactory(AlwaysPassMatcherFactory.class) + SerializableMatcher getOnSuccessMatcher(); + void setOnSuccessMatcher(SerializableMatcher value); + + /** + * Factory for {@link PipelineResult} matchers which always pass. + */ + class AlwaysPassMatcherFactory + implements DefaultValueFactory> { + @Override + public SerializableMatcher create(PipelineOptions options) { + return new AlwaysPassMatcher(); + } + } + + /** + * Matcher which will always pass. + */ + class AlwaysPassMatcher extends BaseMatcher + implements SerializableMatcher { + @Override + public boolean matches(Object o) { + return true; + } + + @Override + public void describeTo(Description description) { + } + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java index 66f7a6a327a8..4eb06d22b87a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java @@ -74,7 +74,7 @@ private ApproximateQuantiles() { * {@code numQuantiles-2} intermediate values, and maximum value, in * sorted order, using the given {@code Comparator} to order values. * To compute traditional {@code N}-tiles, one should use - * {@code ApproximateQuantiles.globally(compareFn, N+1)}. + * {@code ApproximateQuantiles.globally(N+1, compareFn)}. * *

If there are fewer input elements than {@code numQuantiles}, * then the result {@code List} will contain all the input elements, @@ -86,7 +86,7 @@ private ApproximateQuantiles() { *

 {@code
    * PCollection pc = ...;
    * PCollection> quantiles =
-   *     pc.apply(ApproximateQuantiles.globally(stringCompareFn, 11));
+   *     pc.apply(ApproximateQuantiles.globally(11, stringCompareFn));
    * } 
* * @param the type of the elements in the input {@code PCollection} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index f3a7c0360100..8a39c98a16e7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -1661,7 +1661,7 @@ protected SimpleCombineFn(SerializableFunction, V> combiner) { *
 {@code
    * PCollection> salesRecords = ...;
    * PCollection> totalSalesPerPerson =
-   *     salesRecords.apply(Combine.perKey(
+   *     salesRecords.apply(Combine.perKey(
    *         new Sum.SumDoubleFn()));
    * } 
* diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java index cb9c43bd4f68..452ee8ef6637 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java @@ -34,7 +34,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; -import com.google.common.base.Throwables; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableMap; import com.google.common.reflect.TypeParameter; @@ -479,7 +478,7 @@ private void invoke(Method m, throw UserCodeException.wrap(e.getCause()); } catch (IllegalAccessException | IllegalArgumentException e) { // Exception in our code. - throw Throwables.propagate(e); + throw new RuntimeException(e); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java index 4b3afb426ea0..62c09c2158ac 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java @@ -221,7 +221,7 @@ public void processElement(final ProcessContext c) throws Exception { } if (failure.get() != null) { - throw Throwables.propagate(failure.get()); + throw new RuntimeException(failure.get()); } executor.submit(new Runnable() { @@ -246,7 +246,7 @@ public void finishBundle(Context c) throws Exception { // processElement calls have finished. workTickets.acquire(maxParallelism); if (failure.get() != null) { - throw Throwables.propagate(failure.get()); + throw new RuntimeException(failure.get()); } doFn.finishBundle(c); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java index 43f1adf0c0c8..587c041d0654 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java @@ -35,7 +35,7 @@ /** * A UnionCoder encodes RawUnionValues. */ -class UnionCoder extends StandardCoder { +public class UnionCoder extends StandardCoder { // TODO: Think about how to integrate this with a schema object (i.e. // a tuple of tuple tags). /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java index 0fecfdcfd8a8..b12e049a4dab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java @@ -19,7 +19,10 @@ import org.apache.beam.sdk.options.BigQueryOptions; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfigurationExtract; import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; import java.io.IOException; import java.io.Serializable; @@ -30,33 +33,38 @@ public interface BigQueryServices extends Serializable { /** - * Status of a BigQuery job or request. + * Returns a real, mock, or fake {@link JobService}. */ - enum Status { - SUCCEEDED, - FAILED, - UNKNOWN, - } - - /** - * Returns a real, mock, or fake {@link LoadService}. - */ - public LoadService getLoadService(BigQueryOptions bqOptions); + public JobService getJobService(BigQueryOptions bqOptions); /** * An interface for the Cloud BigQuery load service. */ - public interface LoadService { + public interface JobService { /** - * Start a BigQuery load job. + * Starts a BigQuery load job. */ - public void startLoadJob(String jobId, JobConfigurationLoad loadConfig) + void startLoadJob(String jobId, JobConfigurationLoad loadConfig) throws InterruptedException, IOException; /** - * Poll the status of a BigQuery load job. + * Start a BigQuery extract job. + */ + void startExtractJob(String jobId, JobConfigurationExtract extractConfig) + throws InterruptedException, IOException; + + /** + * Start a BigQuery extract job. + */ + void startQueryJob(String jobId, JobConfigurationQuery query, boolean dryRun) + throws IOException, InterruptedException; + + /** + * Waits for the job is Done, and returns the job. + * + *

Returns null if the {@code maxAttempts} retries reached. */ - public Status pollJobStatus(String projectId, String jobId) + Job pollJob(String projectId, String jobId, int maxAttempts) throws InterruptedException, IOException; } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java index 0502c6e9f634..2bfe84f75452 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java @@ -26,10 +26,11 @@ import com.google.api.services.bigquery.Bigquery; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfiguration; +import com.google.api.services.bigquery.model.JobConfigurationExtract; import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.JobStatus; -import com.google.api.services.bigquery.model.TableReference; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; @@ -45,37 +46,34 @@ */ public class BigQueryServicesImpl implements BigQueryServices { - // The maximum number of attempts to execute a load job RPC. - private static final int MAX_LOAD_JOB_RPC_ATTEMPTS = 10; + // The maximum number of attempts to execute a BigQuery RPC. + private static final int MAX_RPC_ATTEMPTS = 10; - // The initial backoff for executing a load job RPC. - private static final long INITIAL_LOAD_JOB_RPC_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1); - // The maximum number of retries to poll the status of a load job. - // It sets to {@code Integer.MAX_VALUE} to block until the BigQuery load job finishes. - private static final int MAX_LOAD_JOB_POLL_RETRIES = Integer.MAX_VALUE; + // The initial backoff for executing a BigQuery RPC. + private static final long INITIAL_RPC_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1); - // The initial backoff for polling the status of a load job. - private static final long INITIAL_LOAD_JOB_POLL_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(60); + // The initial backoff for polling the status of a BigQuery job. + private static final long INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(60); @Override - public LoadService getLoadService(BigQueryOptions options) { - return new LoadServiceImpl(options); + public JobService getJobService(BigQueryOptions options) { + return new JobServiceImpl(options); } @VisibleForTesting - static class LoadServiceImpl implements BigQueryServices.LoadService { - private static final Logger LOG = LoggerFactory.getLogger(LoadServiceImpl.class); + static class JobServiceImpl implements BigQueryServices.JobService { + private static final Logger LOG = LoggerFactory.getLogger(JobServiceImpl.class); private final ApiErrorExtractor errorExtractor; private final Bigquery client; @VisibleForTesting - LoadServiceImpl(Bigquery client) { + JobServiceImpl(Bigquery client) { this.errorExtractor = new ApiErrorExtractor(); this.client = client; } - private LoadServiceImpl(BigQueryOptions options) { + private JobServiceImpl(BigQueryOptions options) { this.errorExtractor = new ApiErrorExtractor(); this.client = Transport.newBigQueryClient(options).build(); } @@ -83,7 +81,7 @@ private LoadServiceImpl(BigQueryOptions options) { /** * {@inheritDoc} * - *

Retries the RPC for at most {@code MAX_LOAD_JOB_RPC_ATTEMPTS} times until it succeeds. + *

Retries the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. * * @throws IOException if it exceeds max RPC retries. */ @@ -91,47 +89,84 @@ private LoadServiceImpl(BigQueryOptions options) { public void startLoadJob( String jobId, JobConfigurationLoad loadConfig) throws InterruptedException, IOException { - BackOff backoff = new AttemptBoundedExponentialBackOff( - MAX_LOAD_JOB_RPC_ATTEMPTS, INITIAL_LOAD_JOB_RPC_BACKOFF_MILLIS); - startLoadJob(jobId, loadConfig, Sleeper.DEFAULT, backoff); + Job job = new Job(); + JobReference jobRef = new JobReference(); + jobRef.setProjectId(loadConfig.getDestinationTable().getProjectId()); + jobRef.setJobId(jobId); + job.setJobReference(jobRef); + JobConfiguration jobConfig = new JobConfiguration(); + jobConfig.setLoad(loadConfig); + job.setConfiguration(jobConfig); + + startJob(job, errorExtractor, client); } /** * {@inheritDoc} * - *

Retries the poll request for at most {@code MAX_LOAD_JOB_POLL_RETRIES} times - * until the job is DONE. + *

Retries the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + * + * @throws IOException if it exceeds max RPC retries. */ @Override - public Status pollJobStatus(String projectId, String jobId) throws InterruptedException { - BackOff backoff = new AttemptBoundedExponentialBackOff( - MAX_LOAD_JOB_POLL_RETRIES, INITIAL_LOAD_JOB_POLL_BACKOFF_MILLIS); - return pollJobStatus(projectId, jobId, Sleeper.DEFAULT, backoff); - } - - @VisibleForTesting - void startLoadJob( - String jobId, - JobConfigurationLoad loadConfig, - Sleeper sleeper, - BackOff backoff) + public void startExtractJob(String jobId, JobConfigurationExtract extractConfig) throws InterruptedException, IOException { - TableReference ref = loadConfig.getDestinationTable(); - String projectId = ref.getProjectId(); + Job job = new Job(); + JobReference jobRef = new JobReference(); + jobRef.setProjectId(extractConfig.getSourceTable().getProjectId()); + jobRef.setJobId(jobId); + job.setJobReference(jobRef); + JobConfiguration jobConfig = new JobConfiguration(); + jobConfig.setExtract(extractConfig); + job.setConfiguration(jobConfig); + + startJob(job, errorExtractor, client); + } + /** + * {@inheritDoc} + * + *

Retries the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + * + * @throws IOException if it exceeds max RPC retries. + */ + @Override + public void startQueryJob(String jobId, JobConfigurationQuery queryConfig, boolean dryRun) + throws IOException, InterruptedException { Job job = new Job(); JobReference jobRef = new JobReference(); - jobRef.setProjectId(projectId); + jobRef.setProjectId(queryConfig.getDestinationTable().getProjectId()); jobRef.setJobId(jobId); job.setJobReference(jobRef); - JobConfiguration config = new JobConfiguration(); - config.setLoad(loadConfig); - job.setConfiguration(config); + JobConfiguration jobConfig = new JobConfiguration(); + jobConfig.setQuery(queryConfig); + jobConfig.setDryRun(dryRun); + job.setConfiguration(jobConfig); + + startJob(job, errorExtractor, client); + } + private static void startJob(Job job, + ApiErrorExtractor errorExtractor, + Bigquery client) throws IOException, InterruptedException { + BackOff backoff = + new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + startJob(job, errorExtractor, client, Sleeper.DEFAULT, backoff); + } + + @VisibleForTesting + static void startJob( + Job job, + ApiErrorExtractor errorExtractor, + Bigquery client, + Sleeper sleeper, + BackOff backoff) + throws InterruptedException, IOException { + JobReference jobRef = job.getJobReference(); Exception lastException = null; do { try { - client.jobs().insert(projectId, job).execute(); + client.jobs().insert(jobRef.getProjectId(), job).execute(); return; // SUCCEEDED } catch (GoogleJsonResponseException e) { if (errorExtractor.itemAlreadyExists(e)) { @@ -149,27 +184,30 @@ void startLoadJob( throw new IOException( String.format( "Unable to insert job: %s, aborting after %d retries.", - jobId, MAX_LOAD_JOB_RPC_ATTEMPTS), + jobRef.getJobId(), MAX_RPC_ATTEMPTS), lastException); } + @Override + public Job pollJob(String projectId, String jobId, int maxAttempts) + throws InterruptedException { + BackOff backoff = new AttemptBoundedExponentialBackOff( + maxAttempts, INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS); + return pollJob(projectId, jobId, Sleeper.DEFAULT, backoff); + } + @VisibleForTesting - Status pollJobStatus( + Job pollJob( String projectId, String jobId, Sleeper sleeper, BackOff backoff) throws InterruptedException { do { try { - JobStatus status = client.jobs().get(projectId, jobId).execute().getStatus(); + Job job = client.jobs().get(projectId, jobId).execute(); + JobStatus status = job.getStatus(); if (status != null && status.getState() != null && status.getState().equals("DONE")) { - if (status.getErrorResult() != null) { - return Status.FAILED; - } else if (status.getErrors() != null && !status.getErrors().isEmpty()) { - return Status.FAILED; - } else { - return Status.SUCCEEDED; - } + return job; } // The job is not DONE, wait longer and retry. } catch (IOException e) { @@ -177,16 +215,16 @@ Status pollJobStatus( LOG.warn("Ignore the error and retry polling job status.", e); } } while (nextBackOff(sleeper, backoff)); - LOG.warn("Unable to poll job status: {}, aborting after {} retries.", - jobId, MAX_LOAD_JOB_POLL_RETRIES); - return Status.UNKNOWN; + LOG.warn("Unable to poll job status: {}, aborting after reached max retries.", jobId); + return null; } /** * Identical to {@link BackOffUtils#next} but without checked IOException. * @throws InterruptedException */ - private boolean nextBackOff(Sleeper sleeper, BackOff backoff) throws InterruptedException { + private static boolean nextBackOff(Sleeper sleeper, BackOff backoff) + throws InterruptedException { try { return BackOffUtils.next(sleeper, backoff); } catch (IOException e) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java index 86a0b5befca8..0493f1c7fa49 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java @@ -37,7 +37,6 @@ import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.common.util.concurrent.MoreExecutors; import org.slf4j.Logger; @@ -275,7 +274,7 @@ public List call() throws IOException { Thread.currentThread().interrupt(); throw new IOException("Interrupted while inserting " + rowsToPublish); } catch (ExecutionException e) { - Throwables.propagate(e.getCause()); + throw new RuntimeException(e.getCause()); } if (!allErrors.isEmpty() && !backoff.atMaxAttempts()) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java index a1d795b606a6..92ff2f08bc37 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java @@ -54,12 +54,6 @@ public class IntervalBoundedExponentialBackOff implements BackOff { private final long initialIntervalMillis; private int currentAttempt; - // BEAM-168: https://issues.apache.org/jira/browse/BEAM-168 - @Deprecated - public IntervalBoundedExponentialBackOff(int maximumIntervalMillis, long initialIntervalMillis) { - this((long) maximumIntervalMillis, initialIntervalMillis); - } - public IntervalBoundedExponentialBackOff(long maximumIntervalMillis, long initialIntervalMillis) { Preconditions.checkArgument( maximumIntervalMillis > 0, "Maximum interval must be greater than zero."); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubApiaryClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubApiaryClient.java new file mode 100644 index 000000000000..f0a90968f510 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubApiaryClient.java @@ -0,0 +1,293 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static com.google.common.base.Preconditions.checkState; + +import org.apache.beam.sdk.options.PubsubOptions; + +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.Pubsub.Builder; +import com.google.api.services.pubsub.model.AcknowledgeRequest; +import com.google.api.services.pubsub.model.ListSubscriptionsResponse; +import com.google.api.services.pubsub.model.ListTopicsResponse; +import com.google.api.services.pubsub.model.ModifyAckDeadlineRequest; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PublishResponse; +import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.api.services.pubsub.model.PullRequest; +import com.google.api.services.pubsub.model.PullResponse; +import com.google.api.services.pubsub.model.ReceivedMessage; +import com.google.api.services.pubsub.model.Subscription; +import com.google.api.services.pubsub.model.Topic; +import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import javax.annotation.Nullable; + +/** + * A Pubsub client using Apiary. + */ +public class PubsubApiaryClient extends PubsubClient { + + public static final PubsubClientFactory FACTORY = new PubsubClientFactory() { + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + Pubsub pubsub = new Builder( + Transport.getTransport(), + Transport.getJsonFactory(), + new ChainingHttpRequestInitializer( + options.getGcpCredential(), + // Do not log 404. It clutters the output and is possibly even required by the caller. + new RetryHttpRequestInitializer(ImmutableList.of(404)))) + .setRootUrl(options.getPubsubRootUrl()) + .setApplicationName(options.getAppName()) + .setGoogleClientRequestInitializer(options.getGoogleApiTrace()) + .build(); + return new PubsubApiaryClient(timestampLabel, idLabel, pubsub); + } + }; + + /** + * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time + * instead. + */ + @Nullable + private final String timestampLabel; + + /** + * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids. + */ + @Nullable + private final String idLabel; + + /** + * Underlying Apiary client. + */ + private Pubsub pubsub; + + @VisibleForTesting + PubsubApiaryClient( + @Nullable String timestampLabel, + @Nullable String idLabel, + Pubsub pubsub) { + this.timestampLabel = timestampLabel; + this.idLabel = idLabel; + this.pubsub = pubsub; + } + + @Override + public void close() { + // Nothing to close. + } + + @Override + public int publish(TopicPath topic, List outgoingMessages) + throws IOException { + List pubsubMessages = new ArrayList<>(outgoingMessages.size()); + for (OutgoingMessage outgoingMessage : outgoingMessages) { + PubsubMessage pubsubMessage = new PubsubMessage().encodeData(outgoingMessage.elementBytes); + + Map attributes = pubsubMessage.getAttributes(); + if ((timestampLabel != null || idLabel != null) && attributes == null) { + attributes = new TreeMap<>(); + pubsubMessage.setAttributes(attributes); + } + + if (timestampLabel != null) { + attributes.put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch)); + } + + if (idLabel != null) { + attributes.put(idLabel, + Hashing.murmur3_128().hashBytes(outgoingMessage.elementBytes).toString()); + } + + pubsubMessages.add(pubsubMessage); + } + PublishRequest request = new PublishRequest().setMessages(pubsubMessages); + PublishResponse response = pubsub.projects() + .topics() + .publish(topic.getPath(), request) + .execute(); + return response.getMessageIds().size(); + } + + @Override + public List pull( + long requestTimeMsSinceEpoch, + SubscriptionPath subscription, + int batchSize, + boolean returnImmediately) throws IOException { + PullRequest request = new PullRequest() + .setReturnImmediately(returnImmediately) + .setMaxMessages(batchSize); + PullResponse response = pubsub.projects() + .subscriptions() + .pull(subscription.getPath(), request) + .execute(); + if (response.getReceivedMessages() == null || response.getReceivedMessages().size() == 0) { + return ImmutableList.of(); + } + List incomingMessages = new ArrayList<>(response.getReceivedMessages().size()); + for (ReceivedMessage message : response.getReceivedMessages()) { + PubsubMessage pubsubMessage = message.getMessage(); + @Nullable Map attributes = pubsubMessage.getAttributes(); + + // Payload. + byte[] elementBytes = pubsubMessage.decodeData(); + + // Timestamp. + long timestampMsSinceEpoch = + extractTimestamp(timestampLabel, message.getMessage().getPublishTime(), attributes); + + // Ack id. + String ackId = message.getAckId(); + checkState(!Strings.isNullOrEmpty(ackId)); + + // Record id, if any. + @Nullable byte[] recordId = null; + if (idLabel != null && attributes != null) { + String recordIdString = attributes.get(idLabel); + if (!Strings.isNullOrEmpty(recordIdString)) { + recordId = recordIdString.getBytes(); + } + } + if (recordId == null) { + recordId = pubsubMessage.getMessageId().getBytes(); + } + + incomingMessages.add(new IncomingMessage(elementBytes, timestampMsSinceEpoch, + requestTimeMsSinceEpoch, ackId, recordId)); + } + + return incomingMessages; + } + + @Override + public void acknowledge(SubscriptionPath subscription, List ackIds) throws IOException { + AcknowledgeRequest request = new AcknowledgeRequest().setAckIds(ackIds); + pubsub.projects() + .subscriptions() + .acknowledge(subscription.getPath(), request) + .execute(); // ignore Empty result. + } + + @Override + public void modifyAckDeadline( + SubscriptionPath subscription, List ackIds, int deadlineSeconds) + throws IOException { + ModifyAckDeadlineRequest request = + new ModifyAckDeadlineRequest().setAckIds(ackIds) + .setAckDeadlineSeconds(deadlineSeconds); + pubsub.projects() + .subscriptions() + .modifyAckDeadline(subscription.getPath(), request) + .execute(); // ignore Empty result. + } + + @Override + public void createTopic(TopicPath topic) throws IOException { + pubsub.projects() + .topics() + .create(topic.getPath(), new Topic()) + .execute(); // ignore Topic result. + } + + @Override + public void deleteTopic(TopicPath topic) throws IOException { + pubsub.projects() + .topics() + .delete(topic.getPath()) + .execute(); // ignore Empty result. + } + + @Override + public List listTopics(ProjectPath project) throws IOException { + ListTopicsResponse response = pubsub.projects() + .topics() + .list(project.getPath()) + .execute(); + if (response.getTopics() == null || response.getTopics().isEmpty()) { + return ImmutableList.of(); + } + List topics = new ArrayList<>(response.getTopics().size()); + for (Topic topic : response.getTopics()) { + topics.add(topicPathFromPath(topic.getName())); + } + return topics; + } + + @Override + public void createSubscription( + TopicPath topic, SubscriptionPath subscription, + int ackDeadlineSeconds) throws IOException { + Subscription request = new Subscription() + .setTopic(topic.getPath()) + .setAckDeadlineSeconds(ackDeadlineSeconds); + pubsub.projects() + .subscriptions() + .create(subscription.getPath(), request) + .execute(); // ignore Subscription result. + } + + @Override + public void deleteSubscription(SubscriptionPath subscription) throws IOException { + pubsub.projects() + .subscriptions() + .delete(subscription.getPath()) + .execute(); // ignore Empty result. + } + + @Override + public List listSubscriptions(ProjectPath project, TopicPath topic) + throws IOException { + ListSubscriptionsResponse response = pubsub.projects() + .subscriptions() + .list(project.getPath()) + .execute(); + if (response.getSubscriptions() == null || response.getSubscriptions().isEmpty()) { + return ImmutableList.of(); + } + List subscriptions = new ArrayList<>(response.getSubscriptions().size()); + for (Subscription subscription : response.getSubscriptions()) { + if (subscription.getTopic().equals(topic.getPath())) { + subscriptions.add(subscriptionPathFromPath(subscription.getName())); + } + } + return subscriptions; + } + + @Override + public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { + Subscription response = pubsub.projects().subscriptions().get(subscription.getPath()).execute(); + return response.getAckDeadlineSeconds(); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java new file mode 100644 index 000000000000..a44329d9f1c6 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java @@ -0,0 +1,489 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; + +import org.apache.beam.sdk.options.PubsubOptions; + +import com.google.api.client.util.DateTime; +import com.google.common.base.Objects; +import com.google.common.base.Strings; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; + +/** + * An (abstract) helper class for talking to Pubsub via an underlying transport. + */ +public abstract class PubsubClient implements Closeable { + /** + * Factory for creating clients. + */ + public interface PubsubClientFactory extends Serializable { + /** + * Construct a new Pubsub client. It should be closed via {@link #close} in order + * to ensure tidy cleanup of underlying netty resources (or use the try-with-resources + * construct). Uses {@code options} to derive pubsub endpoints and application credentials. + * If non-{@literal null}, use {@code timestampLabel} and {@code idLabel} to store custom + * timestamps/ids within message metadata. + */ + PubsubClient newClient( + @Nullable String timestampLabel, + @Nullable String idLabel, + PubsubOptions options) throws IOException; + } + + /** + * Return timestamp as ms-since-unix-epoch corresponding to {@code timestamp}. + * Return {@literal null} if no timestamp could be found. Throw {@link IllegalArgumentException} + * if timestamp cannot be recognized. + */ + @Nullable + private static Long asMsSinceEpoch(@Nullable String timestamp) { + if (Strings.isNullOrEmpty(timestamp)) { + return null; + } + try { + // Try parsing as milliseconds since epoch. Note there is no way to parse a + // string in RFC 3339 format here. + // Expected IllegalArgumentException if parsing fails; we use that to fall back + // to RFC 3339. + return Long.parseLong(timestamp); + } catch (IllegalArgumentException e1) { + // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an + // IllegalArgumentException if parsing fails, and the caller should handle. + return DateTime.parseRfc3339(timestamp).getValue(); + } + } + + /** + * Return the timestamp (in ms since unix epoch) to use for a Pubsub message with {@code + * attributes} and {@code pubsubTimestamp}. + *

If {@code timestampLabel} is non-{@literal null} then the message attributes must contain + * that label, and the value of that label will be taken as the timestamp. + * Otherwise the timestamp will be taken from the Pubsub publish timestamp {@code + * pubsubTimestamp}. Throw {@link IllegalArgumentException} if the timestamp cannot be + * recognized as a ms-since-unix-epoch or RFC3339 time. + * + * @throws IllegalArgumentException + */ + protected static long extractTimestamp( + @Nullable String timestampLabel, + @Nullable String pubsubTimestamp, + @Nullable Map attributes) { + Long timestampMsSinceEpoch; + if (Strings.isNullOrEmpty(timestampLabel)) { + timestampMsSinceEpoch = asMsSinceEpoch(pubsubTimestamp); + checkArgument(timestampMsSinceEpoch != null, + "Cannot interpret PubSub publish timestamp: %s", + pubsubTimestamp); + } else { + String value = attributes == null ? null : attributes.get(timestampLabel); + checkArgument(value != null, + "PubSub message is missing a value for timestamp label %s", + timestampLabel); + timestampMsSinceEpoch = asMsSinceEpoch(value); + checkArgument(timestampMsSinceEpoch != null, + "Cannot interpret value of label %s as timestamp: %s", + timestampLabel, value); + } + return timestampMsSinceEpoch; + } + + /** + * Path representing a cloud project id. + */ + public static class ProjectPath implements Serializable { + private final String path; + + ProjectPath(String path) { + this.path = path; + } + + public String getPath() { + return path; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + ProjectPath that = (ProjectPath) o; + + return path.equals(that.path); + + } + + @Override + public int hashCode() { + return path.hashCode(); + } + + @Override + public String toString() { + return path; + } + } + + public static ProjectPath projectPathFromPath(String path) { + return new ProjectPath(path); + } + + public static ProjectPath projectPathFromId(String projectId) { + return new ProjectPath(String.format("projects/%s", projectId)); + } + + /** + * Path representing a Pubsub subscription. + */ + public static class SubscriptionPath implements Serializable { + private final String path; + + SubscriptionPath(String path) { + this.path = path; + } + + public String getPath() { + return path; + } + + public String getV1Beta1Path() { + String[] splits = path.split("/"); + checkState(splits.length == 4, "Malformed subscription path %s", path); + return String.format("/subscriptions/%s/%s", splits[1], splits[3]); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SubscriptionPath that = (SubscriptionPath) o; + return path.equals(that.path); + } + + @Override + public int hashCode() { + return path.hashCode(); + } + + @Override + public String toString() { + return path; + } + } + + public static SubscriptionPath subscriptionPathFromPath(String path) { + return new SubscriptionPath(path); + } + + public static SubscriptionPath subscriptionPathFromName( + String projectId, String subscriptionName) { + return new SubscriptionPath(String.format("projects/%s/subscriptions/%s", + projectId, subscriptionName)); + } + + /** + * Path representing a Pubsub topic. + */ + public static class TopicPath implements Serializable { + private final String path; + + TopicPath(String path) { + this.path = path; + } + + public String getPath() { + return path; + } + + public String getV1Beta1Path() { + String[] splits = path.split("/"); + checkState(splits.length == 4, "Malformed topic path %s", path); + return String.format("/topics/%s/%s", splits[1], splits[3]); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TopicPath topicPath = (TopicPath) o; + return path.equals(topicPath.path); + } + + @Override + public int hashCode() { + return path.hashCode(); + } + + @Override + public String toString() { + return path; + } + } + + public static TopicPath topicPathFromPath(String path) { + return new TopicPath(path); + } + + public static TopicPath topicPathFromName(String projectId, String topicName) { + return new TopicPath(String.format("projects/%s/topics/%s", projectId, topicName)); + } + + /** + * A message to be sent to Pubsub. + *

NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}. + * Java serialization is never used for non-test clients. + */ + public static class OutgoingMessage implements Serializable { + /** + * Underlying (encoded) element. + */ + public final byte[] elementBytes; + + /** + * Timestamp for element (ms since epoch). + */ + public final long timestampMsSinceEpoch; + + public OutgoingMessage(byte[] elementBytes, long timestampMsSinceEpoch) { + this.elementBytes = elementBytes; + this.timestampMsSinceEpoch = timestampMsSinceEpoch; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + OutgoingMessage that = (OutgoingMessage) o; + + if (timestampMsSinceEpoch != that.timestampMsSinceEpoch) { + return false; + } + return Arrays.equals(elementBytes, that.elementBytes); + + } + + @Override + public int hashCode() { + return Objects.hashCode(Arrays.hashCode(elementBytes), timestampMsSinceEpoch); + } + } + + /** + * A message received from Pubsub. + *

NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}. + * Java serialization is never used for non-test clients. + */ + public static class IncomingMessage implements Serializable { + /** + * Underlying (encoded) element. + */ + public final byte[] elementBytes; + + /** + * Timestamp for element (ms since epoch). Either Pubsub's processing time, + * or the custom timestamp associated with the message. + */ + public final long timestampMsSinceEpoch; + + /** + * Timestamp (in system time) at which we requested the message (ms since epoch). + */ + public final long requestTimeMsSinceEpoch; + + /** + * Id to pass back to Pubsub to acknowledge receipt of this message. + */ + public final String ackId; + + /** + * Id to pass to the runner to distinguish this message from all others. + */ + public final byte[] recordId; + + public IncomingMessage( + byte[] elementBytes, + long timestampMsSinceEpoch, + long requestTimeMsSinceEpoch, + String ackId, + byte[] recordId) { + this.elementBytes = elementBytes; + this.timestampMsSinceEpoch = timestampMsSinceEpoch; + this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch; + this.ackId = ackId; + this.recordId = recordId; + } + + public IncomingMessage withRequestTime(long requestTimeMsSinceEpoch) { + return new IncomingMessage(elementBytes, timestampMsSinceEpoch, requestTimeMsSinceEpoch, + ackId, recordId); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + IncomingMessage that = (IncomingMessage) o; + + if (timestampMsSinceEpoch != that.timestampMsSinceEpoch) { + return false; + } + if (requestTimeMsSinceEpoch != that.requestTimeMsSinceEpoch) { + return false; + } + if (!Arrays.equals(elementBytes, that.elementBytes)) { + return false; + } + if (!ackId.equals(that.ackId)) { + return false; + } + return Arrays.equals(recordId, that.recordId); + } + + @Override + public int hashCode() { + return Objects.hashCode(Arrays.hashCode(elementBytes), timestampMsSinceEpoch, + requestTimeMsSinceEpoch, + ackId, Arrays.hashCode(recordId)); + } + } + + /** + * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages + * published. + * + * @throws IOException + */ + public abstract int publish(TopicPath topic, List outgoingMessages) + throws IOException; + + /** + * Request the next batch of up to {@code batchSize} messages from {@code subscription}. + * Return the received messages, or empty collection if none were available. Does not + * wait for messages to arrive if {@code returnImmediately} is {@literal true}. + * Returned messages will record their request time as {@code requestTimeMsSinceEpoch}. + * + * @throws IOException + */ + public abstract List pull( + long requestTimeMsSinceEpoch, + SubscriptionPath subscription, + int batchSize, + boolean returnImmediately) + throws IOException; + + /** + * Acknowldege messages from {@code subscription} with {@code ackIds}. + * + * @throws IOException + */ + public abstract void acknowledge(SubscriptionPath subscription, List ackIds) + throws IOException; + + /** + * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to + * be {@code deadlineSeconds} from now. + * + * @throws IOException + */ + public abstract void modifyAckDeadline( + SubscriptionPath subscription, List ackIds, + int deadlineSeconds) throws IOException; + + /** + * Create {@code topic}. + * + * @throws IOException + */ + public abstract void createTopic(TopicPath topic) throws IOException; + + /* + * Delete {@code topic}. + * + * @throws IOException + */ + public abstract void deleteTopic(TopicPath topic) throws IOException; + + /** + * Return a list of topics for {@code project}. + * + * @throws IOException + */ + public abstract List listTopics(ProjectPath project) throws IOException; + + /** + * Create {@code subscription} to {@code topic}. + * + * @throws IOException + */ + public abstract void createSubscription( + TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException; + + /** + * Delete {@code subscription}. + * + * @throws IOException + */ + public abstract void deleteSubscription(SubscriptionPath subscription) throws IOException; + + /** + * Return a list of subscriptions for {@code topic} in {@code project}. + * + * @throws IOException + */ + public abstract List listSubscriptions(ProjectPath project, TopicPath topic) + throws IOException; + + /** + * Return the ack deadline, in seconds, for {@code subscription}. + * + * @throws IOException + */ + public abstract int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException; +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java similarity index 68% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java index 66fb61fb6335..b3c1b8f525bc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubGrpcClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java @@ -16,13 +16,15 @@ * limitations under the License. */ -package org.apache.beam.sdk.io; +package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.GcpOptions; +import static com.google.common.base.Preconditions.checkState; + +import org.apache.beam.sdk.options.PubsubOptions; -import com.google.api.client.util.DateTime; import com.google.auth.oauth2.GoogleCredentials; -import com.google.common.base.Preconditions; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.hash.Hashing; import com.google.protobuf.ByteString; @@ -30,6 +32,7 @@ import com.google.pubsub.v1.AcknowledgeRequest; import com.google.pubsub.v1.DeleteSubscriptionRequest; import com.google.pubsub.v1.DeleteTopicRequest; +import com.google.pubsub.v1.GetSubscriptionRequest; import com.google.pubsub.v1.ListSubscriptionsRequest; import com.google.pubsub.v1.ListSubscriptionsResponse; import com.google.pubsub.v1.ListTopicsRequest; @@ -38,11 +41,13 @@ import com.google.pubsub.v1.PublishRequest; import com.google.pubsub.v1.PublishResponse; import com.google.pubsub.v1.PublisherGrpc; +import com.google.pubsub.v1.PublisherGrpc.PublisherBlockingStub; import com.google.pubsub.v1.PubsubMessage; import com.google.pubsub.v1.PullRequest; import com.google.pubsub.v1.PullResponse; import com.google.pubsub.v1.ReceivedMessage; import com.google.pubsub.v1.SubscriberGrpc; +import com.google.pubsub.v1.SubscriberGrpc.SubscriberBlockingStub; import com.google.pubsub.v1.Subscription; import com.google.pubsub.v1.Topic; @@ -56,7 +61,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -68,17 +72,44 @@ /** * A helper class for talking to Pubsub via grpc. */ -public class PubsubGrpcClient implements PubsubClient { +public class PubsubGrpcClient extends PubsubClient { private static final String PUBSUB_ADDRESS = "pubsub.googleapis.com"; private static final int PUBSUB_PORT = 443; private static final List PUBSUB_SCOPES = Collections.singletonList("https://www.googleapis.com/auth/pubsub"); private static final int LIST_BATCH_SIZE = 1000; + private static final int DEFAULT_TIMEOUT_S = 15; + + public static final PubsubClientFactory FACTORY = + new PubsubClientFactory() { + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + ManagedChannel channel = NettyChannelBuilder + .forAddress(PUBSUB_ADDRESS, PUBSUB_PORT) + .negotiationType(NegotiationType.TLS) + .sslContext(GrpcSslContexts.forClient().ciphers(null).build()) + .build(); + // TODO: GcpOptions needs to support building com.google.auth.oauth2.Credentials from the + // various command line options. It currently only supports the older + // com.google.api.client.auth.oauth2.Credentials. + GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); + return new PubsubGrpcClient(timestampLabel, + idLabel, + DEFAULT_TIMEOUT_S, + channel, + credentials, + null /* publisher stub */, + null /* subscriber stub */); + } + }; + /** * Timeout for grpc calls (in s). */ - private static final int TIMEOUT_S = 15; + private final int timeoutSec; /** * Underlying netty channel, or {@literal null} if closed. @@ -104,6 +135,7 @@ public class PubsubGrpcClient implements PubsubClient { @Nullable private final String idLabel; + /** * Cached stubs, or null if not cached. */ @@ -111,35 +143,22 @@ public class PubsubGrpcClient implements PubsubClient { private PublisherGrpc.PublisherBlockingStub cachedPublisherStub; private SubscriberGrpc.SubscriberBlockingStub cachedSubscriberStub; - private PubsubGrpcClient( - @Nullable String timestampLabel, @Nullable String idLabel, - ManagedChannel publisherChannel, GoogleCredentials credentials) { + @VisibleForTesting + PubsubGrpcClient( + @Nullable String timestampLabel, + @Nullable String idLabel, + int timeoutSec, + ManagedChannel publisherChannel, + GoogleCredentials credentials, + PublisherGrpc.PublisherBlockingStub cachedPublisherStub, + SubscriberGrpc.SubscriberBlockingStub cachedSubscriberStub) { this.timestampLabel = timestampLabel; this.idLabel = idLabel; + this.timeoutSec = timeoutSec; this.publisherChannel = publisherChannel; this.credentials = credentials; - } - - /** - * Construct a new Pubsub grpc client. It should be closed via {@link #close} in order - * to ensure tidy cleanup of underlying netty resources. (Or use the try-with-resources - * construct since this class is {@link AutoCloseable}). If non-{@literal null}, use - * {@code timestampLabel} and {@code idLabel} to store custom timestamps/ids within - * message metadata. - */ - public static PubsubGrpcClient newClient( - @Nullable String timestampLabel, @Nullable String idLabel, - GcpOptions options) throws IOException { - ManagedChannel channel = NettyChannelBuilder - .forAddress(PUBSUB_ADDRESS, PUBSUB_PORT) - .negotiationType(NegotiationType.TLS) - .sslContext(GrpcSslContexts.forClient().ciphers(null).build()) - .build(); - // TODO: GcpOptions needs to support building com.google.auth.oauth2.Credentials from the - // various command line options. It currently only supports the older - // com.google.api.client.auth.oauth2.Credentials. - GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); - return new PubsubGrpcClient(timestampLabel, idLabel, channel, credentials); + this.cachedPublisherStub = cachedPublisherStub; + this.cachedSubscriberStub = cachedSubscriberStub; } /** @@ -147,24 +166,34 @@ public static PubsubGrpcClient newClient( */ @Override public void close() { - Preconditions.checkState(publisherChannel != null, "Client has already been closed"); - publisherChannel.shutdown(); - try { - publisherChannel.awaitTermination(TIMEOUT_S, TimeUnit.SECONDS); - } catch (InterruptedException e) { - // Ignore. - Thread.currentThread().interrupt(); + if (publisherChannel == null) { + // Already closed. + return; } - publisherChannel = null; + // Can gc the underlying stubs. cachedPublisherStub = null; cachedSubscriberStub = null; + // Mark the client as having been closed before going further + // in case we have an exception from the channel. + ManagedChannel publisherChannel = this.publisherChannel; + this.publisherChannel = null; + // Gracefully shutdown the channel. + publisherChannel.shutdown(); + if (timeoutSec > 0) { + try { + publisherChannel.awaitTermination(timeoutSec, TimeUnit.SECONDS); + } catch (InterruptedException e) { + // Ignore. + Thread.currentThread().interrupt(); + } + } } /** * Return channel with interceptor for returning credentials. */ private Channel newChannel() throws IOException { - Preconditions.checkState(publisherChannel != null, "PubsubGrpcClient has been closed"); + checkState(publisherChannel != null, "PubsubGrpcClient has been closed"); ClientAuthInterceptor interceptor = new ClientAuthInterceptor(credentials, Executors.newSingleThreadExecutor()); return ClientInterceptors.intercept(publisherChannel, interceptor); @@ -173,25 +202,33 @@ private Channel newChannel() throws IOException { /** * Return a stub for making a publish request with a timeout. */ - private PublisherGrpc.PublisherBlockingStub publisherStub() throws IOException { + private PublisherBlockingStub publisherStub() throws IOException { if (cachedPublisherStub == null) { cachedPublisherStub = PublisherGrpc.newBlockingStub(newChannel()); } - return cachedPublisherStub.withDeadlineAfter(TIMEOUT_S, TimeUnit.SECONDS); + if (timeoutSec > 0) { + return cachedPublisherStub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS); + } else { + return cachedPublisherStub; + } } /** * Return a stub for making a subscribe request with a timeout. */ - private SubscriberGrpc.SubscriberBlockingStub subscriberStub() throws IOException { + private SubscriberBlockingStub subscriberStub() throws IOException { if (cachedSubscriberStub == null) { cachedSubscriberStub = SubscriberGrpc.newBlockingStub(newChannel()); } - return cachedSubscriberStub.withDeadlineAfter(TIMEOUT_S, TimeUnit.SECONDS); + if (timeoutSec > 0) { + return cachedSubscriberStub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS); + } else { + return cachedSubscriberStub; + } } @Override - public int publish(TopicPath topic, Iterable outgoingMessages) + public int publish(TopicPath topic, List outgoingMessages) throws IOException { PublishRequest.Builder request = PublishRequest.newBuilder() .setTopic(topic.getPath()); @@ -208,7 +245,7 @@ public int publish(TopicPath topic, Iterable outgoingMessages) if (idLabel != null) { message.getMutableAttributes() .put(idLabel, - Hashing.murmur3_128().hashBytes(outgoingMessage.elementBytes).toString()); + Hashing.murmur3_128().hashBytes(outgoingMessage.elementBytes).toString()); } request.addMessages(message); @@ -219,13 +256,14 @@ public int publish(TopicPath topic, Iterable outgoingMessages) } @Override - public Collection pull( + public List pull( long requestTimeMsSinceEpoch, SubscriptionPath subscription, - int batchSize) throws IOException { + int batchSize, + boolean returnImmediately) throws IOException { PullRequest request = PullRequest.newBuilder() .setSubscription(subscription.getPath()) - .setReturnImmediately(true) + .setReturnImmediately(returnImmediately) .setMaxMessages(batchSize) .build(); PullResponse response = subscriberStub().pull(request); @@ -235,41 +273,24 @@ public Collection pull( List incomingMessages = new ArrayList<>(response.getReceivedMessagesCount()); for (ReceivedMessage message : response.getReceivedMessagesList()) { PubsubMessage pubsubMessage = message.getMessage(); - Map attributes = pubsubMessage.getAttributes(); + @Nullable Map attributes = pubsubMessage.getAttributes(); // Payload. byte[] elementBytes = pubsubMessage.getData().toByteArray(); // Timestamp. - // Start with Pubsub processing time. + String pubsubTimestampString = null; Timestamp timestampProto = pubsubMessage.getPublishTime(); - long timestampMsSinceEpoch = timestampProto.getSeconds() + timestampProto.getNanos() / 1000L; - if (timestampLabel != null && attributes != null) { - String timestampString = attributes.get(timestampLabel); - if (timestampString != null && !timestampString.isEmpty()) { - try { - // Try parsing as milliseconds since epoch. Note there is no way to parse a - // string in RFC 3339 format here. - // Expected IllegalArgumentException if parsing fails; we use that to fall back - // to RFC 3339. - timestampMsSinceEpoch = Long.parseLong(timestampString); - } catch (IllegalArgumentException e1) { - try { - // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an - // IllegalArgumentException if parsing fails, and the caller should handle. - timestampMsSinceEpoch = DateTime.parseRfc3339(timestampString).getValue(); - } catch (IllegalArgumentException e2) { - // Fallback to Pubsub processing time. - } - } - } - // else: fallback to Pubsub processing time. + if (timestampProto != null) { + pubsubTimestampString = String.valueOf(timestampProto.getSeconds() + + timestampProto.getNanos() / 1000L); } - // else: fallback to Pubsub processing time. + long timestampMsSinceEpoch = + extractTimestamp(timestampLabel, pubsubTimestampString, attributes); // Ack id. String ackId = message.getAckId(); - Preconditions.checkState(ackId != null && !ackId.isEmpty()); + checkState(!Strings.isNullOrEmpty(ackId)); // Record id, if any. @Nullable byte[] recordId = null; @@ -284,13 +305,13 @@ public Collection pull( } incomingMessages.add(new IncomingMessage(elementBytes, timestampMsSinceEpoch, - requestTimeMsSinceEpoch, ackId, recordId)); + requestTimeMsSinceEpoch, ackId, recordId)); } return incomingMessages; } @Override - public void acknowledge(SubscriptionPath subscription, Iterable ackIds) + public void acknowledge(SubscriptionPath subscription, List ackIds) throws IOException { AcknowledgeRequest request = AcknowledgeRequest.newBuilder() .setSubscription(subscription.getPath()) @@ -301,8 +322,7 @@ public void acknowledge(SubscriptionPath subscription, Iterable ackIds) @Override public void modifyAckDeadline( - SubscriptionPath subscription, Iterable ackIds, int - deadlineSeconds) + SubscriptionPath subscription, List ackIds, int deadlineSeconds) throws IOException { ModifyAckDeadlineRequest request = ModifyAckDeadlineRequest.newBuilder() @@ -330,7 +350,7 @@ public void deleteTopic(TopicPath topic) throws IOException { } @Override - public Collection listTopics(ProjectPath project) throws IOException { + public List listTopics(ProjectPath project) throws IOException { ListTopicsRequest.Builder request = ListTopicsRequest.newBuilder() .setProject(project.getPath()) @@ -342,7 +362,7 @@ public Collection listTopics(ProjectPath project) throws IOException List topics = new ArrayList<>(response.getTopicsCount()); while (true) { for (Topic topic : response.getTopicsList()) { - topics.add(new TopicPath(topic.getName())); + topics.add(topicPathFromPath(topic.getName())); } if (response.getNextPageToken().isEmpty()) { break; @@ -375,7 +395,7 @@ public void deleteSubscription(SubscriptionPath subscription) throws IOException } @Override - public Collection listSubscriptions(ProjectPath project, TopicPath topic) + public List listSubscriptions(ProjectPath project, TopicPath topic) throws IOException { ListSubscriptionsRequest.Builder request = ListSubscriptionsRequest.newBuilder() @@ -389,7 +409,7 @@ public Collection listSubscriptions(ProjectPath project, Topic while (true) { for (Subscription subscription : response.getSubscriptionsList()) { if (subscription.getTopic().equals(topic.getPath())) { - subscriptions.add(new SubscriptionPath(subscription.getName())); + subscriptions.add(subscriptionPathFromPath(subscription.getName())); } } if (response.getNextPageToken().isEmpty()) { @@ -400,4 +420,14 @@ public Collection listSubscriptions(ProjectPath project, Topic } return subscriptions; } + + @Override + public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { + GetSubscriptionRequest request = + GetSubscriptionRequest.newBuilder() + .setSubscription(subscription.getPath()) + .build(); + Subscription response = subscriberStub().getSubscription(request); + return response.getAckDeadlineSeconds(); + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java new file mode 100644 index 000000000000..4a47c3070776 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import org.apache.beam.sdk.options.PubsubOptions; + +import com.google.common.annotations.VisibleForTesting; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; + +/** + * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for + * testing {@link #publish}, {@link #pull}, {@link #acknowledge} and {@link #modifyAckDeadline} + * methods. + */ +public class PubsubTestClient extends PubsubClient { + public static PubsubClientFactory createFactoryForPublish( + final TopicPath expectedTopic, + final Set expectedOutgoingMessages) { + return new PubsubClientFactory() { + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + return new PubsubTestClient(expectedTopic, null, 0, expectedOutgoingMessages, null); + } + }; + } + + public static PubsubClientFactory createFactoryForPull( + @Nullable final SubscriptionPath expectedSubscription, + final int ackTimeoutSec, + @Nullable final List expectedIncomingMessages) { + return new PubsubClientFactory() { + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + return new PubsubTestClient(null, expectedSubscription, ackTimeoutSec, + null, expectedIncomingMessages); + } + }; + } + + /** + * Only publish calls for this topic are allowed. + */ + @Nullable + private TopicPath expectedTopic; + /** + * Only pull calls for this subscription are allowed. + */ + @Nullable + private SubscriptionPath expectedSubscription; + + /** + * Timeout to simulate. + */ + private int ackTimeoutSec; + + /** + * Messages yet to seen in a {@link #publish} call. + */ + @Nullable + private Set remainingExpectedOutgoingMessages; + + /** + * Messages waiting to be received by a {@link #pull} call. + */ + @Nullable + private List remainingPendingIncomingMessages; + + /** + * Messages which have been returned from a {@link #pull} call and + * not yet ACKed by an {@link #acknowledge} call. + */ + private Map pendingAckIncommingMessages; + + /** + * When above messages are due to have their ACK deadlines expire. + */ + private Map ackDeadline; + + /** + * Current time. + */ + private long nowMsSinceEpoch; + + @VisibleForTesting + PubsubTestClient( + @Nullable TopicPath expectedTopic, + @Nullable SubscriptionPath expectedSubscription, + int ackTimeoutSec, + @Nullable Set expectedOutgoingMessages, + @Nullable List expectedIncomingMessages) { + this.expectedTopic = expectedTopic; + this.expectedSubscription = expectedSubscription; + this.ackTimeoutSec = ackTimeoutSec; + + this.remainingExpectedOutgoingMessages = expectedOutgoingMessages; + this.remainingPendingIncomingMessages = expectedIncomingMessages; + + this.pendingAckIncommingMessages = new HashMap<>(); + this.ackDeadline = new HashMap<>(); + this.nowMsSinceEpoch = Long.MIN_VALUE; + } + + /** + * Advance wall-clock time to {@code newNowMsSinceEpoch}. This will simulate Pubsub expiring + * outstanding ACKs. + */ + public void advanceTo(long newNowMsSinceEpoch) { + checkArgument(newNowMsSinceEpoch >= nowMsSinceEpoch, + "Cannot advance time backwards from %d to %d", nowMsSinceEpoch, + newNowMsSinceEpoch); + nowMsSinceEpoch = newNowMsSinceEpoch; + // Any messages who's ACKs timed out are available for re-pulling. + Iterator> deadlineItr = ackDeadline.entrySet().iterator(); + while (deadlineItr.hasNext()) { + Map.Entry entry = deadlineItr.next(); + if (entry.getValue() <= nowMsSinceEpoch) { + remainingPendingIncomingMessages.add(pendingAckIncommingMessages.remove(entry.getKey())); + deadlineItr.remove(); + } + } + } + + @Override + public void close() { + if (remainingExpectedOutgoingMessages != null) { + checkState(this.remainingExpectedOutgoingMessages.isEmpty(), + "Failed to pull %d messages", this.remainingExpectedOutgoingMessages.size()); + remainingExpectedOutgoingMessages = null; + } + if (remainingPendingIncomingMessages != null) { + checkState(remainingPendingIncomingMessages.isEmpty(), + "Failed to publish %d messages", remainingPendingIncomingMessages.size()); + checkState(pendingAckIncommingMessages.isEmpty(), + "Failed to ACK %d messages", pendingAckIncommingMessages.size()); + checkState(ackDeadline.isEmpty(), + "Failed to ACK %d messages", ackDeadline.size()); + remainingPendingIncomingMessages = null; + pendingAckIncommingMessages = null; + ackDeadline = null; + } + } + + @Override + public int publish( + TopicPath topic, List outgoingMessages) throws IOException { + checkNotNull(expectedTopic, "Missing expected topic"); + checkNotNull(remainingExpectedOutgoingMessages, "Missing expected outgoing messages"); + checkState(topic.equals(expectedTopic), "Topic %s does not match expected %s", topic, + expectedTopic); + for (OutgoingMessage outgoingMessage : outgoingMessages) { + checkState(remainingExpectedOutgoingMessages.remove(outgoingMessage), + "Unexpeced outgoing message %s", outgoingMessage); + } + return outgoingMessages.size(); + } + + @Override + public List pull( + long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize, + boolean returnImmediately) throws IOException { + checkState(requestTimeMsSinceEpoch == nowMsSinceEpoch, + "Simulated time %d does not match requset time %d", nowMsSinceEpoch, + requestTimeMsSinceEpoch); + checkNotNull(expectedSubscription, "Missing expected subscription"); + checkNotNull(remainingPendingIncomingMessages, "Missing expected incoming messages"); + checkState(subscription.equals(expectedSubscription), + "Subscription %s does not match expected %s", subscription, expectedSubscription); + checkState(returnImmediately, "PubsubTestClient only supports returning immediately"); + + List incomingMessages = new ArrayList<>(); + Iterator pendItr = remainingPendingIncomingMessages.iterator(); + while (pendItr.hasNext()) { + IncomingMessage incomingMessage = pendItr.next(); + pendItr.remove(); + IncomingMessage incomingMessageWithRequestTime = + incomingMessage.withRequestTime(requestTimeMsSinceEpoch); + incomingMessages.add(incomingMessageWithRequestTime); + pendingAckIncommingMessages.put(incomingMessageWithRequestTime.ackId, + incomingMessageWithRequestTime); + ackDeadline.put(incomingMessageWithRequestTime.ackId, + requestTimeMsSinceEpoch + ackTimeoutSec * 1000); + if (incomingMessages.size() >= batchSize) { + break; + } + } + return incomingMessages; + } + + @Override + public void acknowledge( + SubscriptionPath subscription, + List ackIds) throws IOException { + checkNotNull(expectedSubscription, "Missing expected subscription"); + checkNotNull(remainingPendingIncomingMessages, "Missing expected incoming messages"); + checkState(subscription.equals(expectedSubscription), + "Subscription %s does not match expected %s", subscription, expectedSubscription); + + for (String ackId : ackIds) { + checkState(ackDeadline.remove(ackId) != null, + "No message with ACK id %s is outstanding", ackId); + checkState(pendingAckIncommingMessages.remove(ackId) != null, + "No message with ACK id %s is outstanding", ackId); + } + } + + @Override + public void modifyAckDeadline( + SubscriptionPath subscription, List ackIds, int deadlineSeconds) throws IOException { + checkNotNull(expectedSubscription, "Missing expected subscription"); + checkNotNull(remainingPendingIncomingMessages, "Missing expected incoming messages"); + checkState(subscription.equals(expectedSubscription), + "Subscription %s does not match expected %s", subscription, expectedSubscription); + + for (String ackId : ackIds) { + checkState(ackDeadline.remove(ackId) != null, + "No message with ACK id %s is outstanding", ackId); + checkState(pendingAckIncommingMessages.containsKey(ackId), + "No message with ACK id %s is outstanding", ackId); + ackDeadline.put(ackId, nowMsSinceEpoch + deadlineSeconds * 1000); + } + } + + @Override + public void createTopic(TopicPath topic) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteTopic(TopicPath topic) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public List listTopics(ProjectPath project) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void createSubscription( + TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteSubscription(SubscriptionPath subscription) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public List listSubscriptions( + ProjectPath project, TopicPath topic) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { + return ackTimeoutSec; + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java new file mode 100644 index 000000000000..4eeedf65a149 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.PCollectionView; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning + * them via the {@link #processElementInReadyWindows(WindowedValue)}. + */ +class PushbackSideInputDoFnRunner implements DoFnRunner { + private final DoFnRunner underlying; + private final Collection> views; + private final ReadyCheckingSideInputReader sideInputReader; + + private Set notReadyWindows; + + public static PushbackSideInputDoFnRunner create( + DoFnRunner underlying, + Collection> views, + ReadyCheckingSideInputReader sideInputReader) { + return new PushbackSideInputDoFnRunner<>(underlying, views, sideInputReader); + } + + private PushbackSideInputDoFnRunner( + DoFnRunner underlying, + Collection> views, + ReadyCheckingSideInputReader sideInputReader) { + this.underlying = underlying; + this.views = views; + this.sideInputReader = sideInputReader; + } + + @Override + public void startBundle() { + notReadyWindows = new HashSet<>(); + underlying.startBundle(); + } + + /** + * Call the underlying {@link DoFnRunner#processElement(WindowedValue)} for the provided element + * for each window the element is in that is ready. + * + * @param elem the element to process in all ready windows + * @return each element that could not be processed because it requires a side input window + * that is not ready. + */ + public Iterable> processElementInReadyWindows(WindowedValue elem) { + if (views.isEmpty()) { + processElement(elem); + return Collections.emptyList(); + } + ImmutableList.Builder> pushedBack = ImmutableList.builder(); + for (WindowedValue windowElem : elem.explodeWindows()) { + BoundedWindow mainInputWindow = Iterables.getOnlyElement(windowElem.getWindows()); + boolean isReady = !notReadyWindows.contains(mainInputWindow); + for (PCollectionView view : views) { + BoundedWindow sideInputWindow = + view.getWindowingStrategyInternal() + .getWindowFn() + .getSideInputWindow(mainInputWindow); + if (!sideInputReader.isReady(view, sideInputWindow)) { + isReady = false; + break; + } + } + if (isReady) { + processElement(windowElem); + } else { + notReadyWindows.add(mainInputWindow); + pushedBack.add(windowElem); + } + } + return pushedBack.build(); + } + + @Override + public void processElement(WindowedValue elem) { + underlying.processElement(elem); + } + + /** + * Call the underlying {@link DoFnRunner#finishBundle()}. + */ + @Override + public void finishBundle() { + notReadyWindows = null; + underlying.finishBundle(); + } +} + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReadyCheckingSideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReadyCheckingSideInputReader.java new file mode 100644 index 000000000000..cb38a55a0da4 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReadyCheckingSideInputReader.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.PCollectionView; + +/** + * A {@link SideInputReader} that allows callers to check to see if a {@link PCollectionView} has + * had its contents set in a window. + */ +public interface ReadyCheckingSideInputReader extends SideInputReader { + /** + * Returns true if the {@link PCollectionView} is ready in the provided {@link BoundedWindow}. + */ + boolean isReady(PCollectionView view, BoundedWindow window); +} + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java index 9082ce375b7f..6daecdb6ce60 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java @@ -105,7 +105,11 @@ private static ApiComponents apiComponentsFromUrl(String urlString) { /** * Returns a Pubsub client builder using the specified {@link PubsubOptions}. + * + * @deprecated Use an appropriate + * {@link org.apache.beam.sdk.util.PubsubClient.PubsubClientFactory} */ + @Deprecated public static Pubsub.Builder newPubsubClient(PubsubOptions options) { return new Pubsub.Builder(getTransport(), getJsonFactory(), diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java index 63ff22c63cc5..7998fc70b7df 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.TableRowJsonCoder; +import org.apache.beam.sdk.io.BigQueryIO.Status; import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.options.BigQueryOptions; @@ -39,16 +40,21 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.BigQueryServices; -import org.apache.beam.sdk.util.BigQueryServices.Status; import org.apache.beam.sdk.util.CoderUtils; import com.google.api.client.util.Data; +import com.google.api.services.bigquery.model.ErrorProto; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfigurationExtract; import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import com.google.api.services.bigquery.model.JobStatus; import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.hamcrest.Matchers; import org.junit.Assert; @@ -66,6 +72,7 @@ import java.io.File; import java.io.FileFilter; import java.io.IOException; +import java.util.Map; /** * Tests for BigQueryIO. @@ -73,23 +80,28 @@ @RunWith(JUnit4.class) public class BigQueryIOTest { + // Status.UNKNOWN maps to null + private static final Map JOB_STATUS_MAP = ImmutableMap.of( + Status.SUCCEEDED, new Job().setStatus(new JobStatus()), + Status.FAILED, new Job().setStatus(new JobStatus().setErrorResult(new ErrorProto()))); + private static class FakeBigQueryServices implements BigQueryServices { - private Object[] startLoadJobReturns; + private Object[] startJobReturns; private Object[] pollJobStatusReturns; /** - * Sets the return values for the mock {@link LoadService#startLoadJob}. + * Sets the return values for the mock {@link JobService#startLoadJob}. * *

Throws if the {@link Object} is a {@link Exception}, returns otherwise. */ private FakeBigQueryServices startLoadJobReturns(Object... startLoadJobReturns) { - this.startLoadJobReturns = startLoadJobReturns; + this.startJobReturns = startLoadJobReturns; return this; } /** - * Sets the return values for the mock {@link LoadService#pollJobStatus}. + * Sets the return values for the mock {@link JobService#pollJobStatus}. * *

Throws if the {@link Object} is a {@link Exception}, returns otherwise. */ @@ -99,19 +111,19 @@ private FakeBigQueryServices pollJobStatusReturns(Object... pollJobStatusReturns } @Override - public LoadService getLoadService(BigQueryOptions bqOptions) { - return new FakeLoadService(startLoadJobReturns, pollJobStatusReturns); + public JobService getJobService(BigQueryOptions bqOptions) { + return new FakeLoadService(startJobReturns, pollJobStatusReturns); } - private static class FakeLoadService implements BigQueryServices.LoadService { + private static class FakeLoadService implements BigQueryServices.JobService { - private Object[] startLoadJobReturns; + private Object[] startJobReturns; private Object[] pollJobStatusReturns; private int startLoadJobCallsCount; private int pollJobStatusCallsCount; public FakeLoadService(Object[] startLoadJobReturns, Object[] pollJobStatusReturns) { - this.startLoadJobReturns = startLoadJobReturns; + this.startJobReturns = startLoadJobReturns; this.pollJobStatusReturns = pollJobStatusReturns; this.startLoadJobCallsCount = 0; this.pollJobStatusCallsCount = 0; @@ -120,35 +132,52 @@ public FakeLoadService(Object[] startLoadJobReturns, Object[] pollJobStatusRetur @Override public void startLoadJob(String jobId, JobConfigurationLoad loadConfig) throws InterruptedException, IOException { - if (startLoadJobCallsCount < startLoadJobReturns.length) { - Object ret = startLoadJobReturns[startLoadJobCallsCount++]; - if (ret instanceof IOException) { - throw (IOException) ret; + startJob(); + } + + @Override + public void startExtractJob(String jobId, JobConfigurationExtract extractConfig) + throws InterruptedException, IOException { + startJob(); + } + + @Override + public void startQueryJob(String jobId, JobConfigurationQuery query, boolean dryRun) + throws IOException, InterruptedException { + startJob(); + } + + @Override + public Job pollJob(String projectId, String jobId, int maxAttemps) + throws InterruptedException { + if (pollJobStatusCallsCount < pollJobStatusReturns.length) { + Object ret = pollJobStatusReturns[pollJobStatusCallsCount++]; + if (ret instanceof Status) { + return JOB_STATUS_MAP.get(ret); } else if (ret instanceof InterruptedException) { throw (InterruptedException) ret; } else { - return; + throw new RuntimeException("Unexpected return type: " + ret.getClass()); } } else { throw new RuntimeException( - "Exceeded expected number of calls: " + startLoadJobReturns.length); + "Exceeded expected number of calls: " + pollJobStatusReturns.length); } } - @Override - public Status pollJobStatus(String projectId, String jobId) throws InterruptedException { - if (pollJobStatusCallsCount < pollJobStatusReturns.length) { - Object ret = pollJobStatusReturns[pollJobStatusCallsCount++]; - if (ret instanceof Status) { - return (Status) ret; + private void startJob() throws IOException, InterruptedException { + if (startLoadJobCallsCount < startJobReturns.length) { + Object ret = startJobReturns[startLoadJobCallsCount++]; + if (ret instanceof IOException) { + throw (IOException) ret; } else if (ret instanceof InterruptedException) { throw (InterruptedException) ret; } else { - throw new RuntimeException("Unexpected return type: " + ret.getClass()); + return; } } else { throw new RuntimeException( - "Exceeded expected number of calls: " + pollJobStatusReturns.length); + "Exceeded expected number of calls: " + startJobReturns.length); } } } @@ -160,7 +189,7 @@ public Status pollJobStatus(String projectId, String jobId) throws InterruptedEx @Rule public TemporaryFolder testFolder = new TemporaryFolder(); @Mock - public BigQueryServices.LoadService mockBqLoadService; + public BigQueryServices.JobService mockBqLoadService; private BigQueryOptions bqOptions; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java index 1e5bf5157b05..eaf452d8bafa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java @@ -24,22 +24,13 @@ import org.apache.beam.sdk.transforms.display.DisplayData; -import com.google.api.client.testing.http.FixedClock; -import com.google.api.client.util.Clock; -import com.google.api.services.pubsub.model.PubsubMessage; - import org.joda.time.Duration; -import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.HashMap; - -import javax.annotation.Nullable; - /** * Tests for PubsubIO Read and Write transforms. */ @@ -90,154 +81,6 @@ public void testTopicValidationTooLong() throws Exception { .toString()); } - /** - * Helper function that creates a {@link PubsubMessage} with the given timestamp registered as - * an attribute with the specified label. - * - *

If {@code label} is {@code null}, then the attributes are {@code null}. - * - *

Else, if {@code timestamp} is {@code null}, then attributes are present but have no key for - * the label. - */ - private static PubsubMessage messageWithTimestamp( - @Nullable String label, @Nullable String timestamp) { - PubsubMessage message = new PubsubMessage(); - if (label == null) { - message.setAttributes(null); - return message; - } - - message.setAttributes(new HashMap()); - - if (timestamp == null) { - return message; - } - - message.getAttributes().put(label, timestamp); - return message; - } - - /** - * Helper function that parses the given string to a timestamp through the PubSubIO plumbing. - */ - private static Instant parseTimestamp(@Nullable String timestamp) { - PubsubMessage message = messageWithTimestamp("mylabel", timestamp); - return PubsubIO.assignMessageTimestamp(message, "mylabel", Clock.SYSTEM); - } - - @Test - public void noTimestampLabelReturnsNow() { - final long time = 987654321L; - Instant timestamp = PubsubIO.assignMessageTimestamp( - messageWithTimestamp(null, null), null, new FixedClock(time)); - - assertEquals(new Instant(time), timestamp); - } - - @Test - public void timestampLabelWithNullAttributesThrowsError() { - PubsubMessage message = messageWithTimestamp(null, null); - thrown.expect(RuntimeException.class); - thrown.expectMessage("PubSub message is missing a timestamp in label: myLabel"); - - PubsubIO.assignMessageTimestamp(message, "myLabel", Clock.SYSTEM); - } - - @Test - public void timestampLabelSetWithMissingAttributeThrowsError() { - PubsubMessage message = messageWithTimestamp("notMyLabel", "ignored"); - thrown.expect(RuntimeException.class); - thrown.expectMessage("PubSub message is missing a timestamp in label: myLabel"); - - PubsubIO.assignMessageTimestamp(message, "myLabel", Clock.SYSTEM); - } - - @Test - public void timestampLabelParsesMillisecondsSinceEpoch() { - Long millis = 1446162101123L; - assertEquals(new Instant(millis), parseTimestamp(millis.toString())); - } - - @Test - public void timestampLabelParsesRfc3339Seconds() { - String rfc3339 = "2015-10-29T23:41:41Z"; - assertEquals(Instant.parse(rfc3339), parseTimestamp(rfc3339)); - } - - @Test - public void timestampLabelParsesRfc3339Tenths() { - String rfc3339tenths = "2015-10-29T23:41:41.1Z"; - assertEquals(Instant.parse(rfc3339tenths), parseTimestamp(rfc3339tenths)); - } - - @Test - public void timestampLabelParsesRfc3339Hundredths() { - String rfc3339hundredths = "2015-10-29T23:41:41.12Z"; - assertEquals(Instant.parse(rfc3339hundredths), parseTimestamp(rfc3339hundredths)); - } - - @Test - public void timestampLabelParsesRfc3339Millis() { - String rfc3339millis = "2015-10-29T23:41:41.123Z"; - assertEquals(Instant.parse(rfc3339millis), parseTimestamp(rfc3339millis)); - } - - @Test - public void timestampLabelParsesRfc3339Micros() { - String rfc3339micros = "2015-10-29T23:41:41.123456Z"; - assertEquals(Instant.parse(rfc3339micros), parseTimestamp(rfc3339micros)); - // Note: micros part 456/1000 is dropped. - assertEquals(Instant.parse("2015-10-29T23:41:41.123Z"), parseTimestamp(rfc3339micros)); - } - - @Test - public void timestampLabelParsesRfc3339MicrosRounding() { - String rfc3339micros = "2015-10-29T23:41:41.123999Z"; - assertEquals(Instant.parse(rfc3339micros), parseTimestamp(rfc3339micros)); - // Note: micros part 999/1000 is dropped, not rounded up. - assertEquals(Instant.parse("2015-10-29T23:41:41.123Z"), parseTimestamp(rfc3339micros)); - } - - @Test - public void timestampLabelWithInvalidFormatThrowsError() { - thrown.expect(NumberFormatException.class); - parseTimestamp("not-a-timestamp"); - } - - @Test - public void timestampLabelWithInvalidFormat2ThrowsError() { - thrown.expect(NumberFormatException.class); - parseTimestamp("null"); - } - - @Test - public void timestampLabelWithInvalidFormat3ThrowsError() { - thrown.expect(NumberFormatException.class); - parseTimestamp("2015-10"); - } - - @Test - public void timestampLabelParsesRfc3339WithSmallYear() { - // Google and JodaTime agree on dates after 1582-10-15, when the Gregorian Calendar was adopted - // This is therefore a "small year" until this difference is reconciled. - String rfc3339SmallYear = "1582-10-15T01:23:45.123Z"; - assertEquals(Instant.parse(rfc3339SmallYear), parseTimestamp(rfc3339SmallYear)); - } - - @Test - public void timestampLabelParsesRfc3339WithLargeYear() { - // Year 9999 in range. - String rfc3339LargeYear = "9999-10-29T23:41:41.123999Z"; - assertEquals(Instant.parse(rfc3339LargeYear), parseTimestamp(rfc3339LargeYear)); - } - - @Test - public void timestampLabelRfc3339WithTooLargeYearThrowsError() { - thrown.expect(NumberFormatException.class); - // Year 10000 out of range. - parseTimestamp("10000-10-29T23:41:41.123999Z"); - } - @Test public void testReadDisplayData() { String topic = "projects/project/topics/topic"; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java index 9460e13d2c84..8af4ff25bb06 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -31,6 +32,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestRule; @@ -38,7 +41,9 @@ import org.junit.runners.JUnit4; import java.util.Arrays; +import java.util.Date; import java.util.List; +import java.util.UUID; /** Tests for {@link TestPipeline}. */ @RunWith(JUnit4.class) @@ -116,4 +121,52 @@ public TestPipeline p() { return TestPipeline.create(); } } + + @Test + public void testMatcherSerializationDeserialization() { + TestPipelineOptions opts = PipelineOptionsFactory.as(TestPipelineOptions.class); + SerializableMatcher m1 = new TestMatcher(); + SerializableMatcher m2 = new TestMatcher(); + + opts.setOnCreateMatcher(m1); + opts.setOnSuccessMatcher(m2); + + String[] arr = TestPipeline.convertToArgs(opts); + TestPipelineOptions newOpts = PipelineOptionsFactory.fromArgs(arr) + .as(TestPipelineOptions.class); + + assertEquals(m1, newOpts.getOnCreateMatcher()); + assertEquals(m2, newOpts.getOnSuccessMatcher()); + } + + /** + * TestMatcher is a matcher designed for testing matcher serialization/deserialization. + */ + public static class TestMatcher extends BaseMatcher + implements SerializableMatcher { + private final UUID uuid = UUID.randomUUID(); + @Override + public boolean matches(Object o) { + return true; + } + + @Override + public void describeTo(Description description) { + description.appendText(String.format("%tL", new Date())); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof TestMatcher)) { + return false; + } + TestMatcher other = (TestMatcher) obj; + return other.uuid.equals(uuid); + } + + @Override + public int hashCode() { + return uuid.hashCode(); + } + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java index 74a2da2198ec..238deede2b86 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryServicesImplTest.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; +import org.apache.beam.sdk.util.BigQueryServicesImpl.JobServiceImpl; import com.google.api.client.googleapis.json.GoogleJsonError; import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; @@ -40,8 +41,10 @@ import com.google.api.services.bigquery.model.ErrorProto; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.JobStatus; import com.google.api.services.bigquery.model.TableReference; +import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.collect.ImmutableList; import org.junit.Before; @@ -91,11 +94,16 @@ public LowLevelHttpResponse execute() throws IOException { } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#startLoadJob} succeeds. + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds. */ @Test public void testStartLoadJobSucceeds() throws IOException, InterruptedException { Job testJob = new Job(); + JobReference jobRef = new JobReference(); + jobRef.setJobId("jobId"); + jobRef.setProjectId("projectId"); + testJob.setJobReference(jobRef); + when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); when(response.getStatusCode()).thenReturn(200); when(response.getContent()).thenReturn(toStream(testJob)); @@ -108,20 +116,24 @@ public void testStartLoadJobSucceeds() throws IOException, InterruptedException Sleeper sleeper = new FastNanoClockAndSleeper(); BackOff backoff = new AttemptBoundedExponentialBackOff( 5 /* attempts */, 1000 /* initialIntervalMillis */); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - loadService.startLoadJob("jobId", loadConfig, sleeper, backoff); + JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); verify(response, times(1)).getContentType(); } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#startLoadJob} succeeds + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds * with an already exist job. */ @Test public void testStartLoadJobSucceedsAlreadyExists() throws IOException, InterruptedException { + Job testJob = new Job(); + JobReference jobRef = new JobReference(); + jobRef.setJobId("jobId"); + jobRef.setProjectId("projectId"); + testJob.setJobReference(jobRef); + when(response.getStatusCode()).thenReturn(409); // 409 means already exists TableReference ref = new TableReference(); @@ -132,9 +144,7 @@ public void testStartLoadJobSucceedsAlreadyExists() throws IOException, Interrup Sleeper sleeper = new FastNanoClockAndSleeper(); BackOff backoff = new AttemptBoundedExponentialBackOff( 5 /* attempts */, 1000 /* initialIntervalMillis */); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - loadService.startLoadJob("jobId", loadConfig, sleeper, backoff); + JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); @@ -142,11 +152,15 @@ public void testStartLoadJobSucceedsAlreadyExists() throws IOException, Interrup } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#startLoadJob} succeeds with a retry. + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#startLoadJob} succeeds with a retry. */ @Test public void testStartLoadJobRetry() throws IOException, InterruptedException { Job testJob = new Job(); + JobReference jobRef = new JobReference(); + jobRef.setJobId("jobId"); + jobRef.setProjectId("projectId"); + testJob.setJobReference(jobRef); // First response is 403 rate limited, second response has valid payload. when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); @@ -163,19 +177,18 @@ public void testStartLoadJobRetry() throws IOException, InterruptedException { Sleeper sleeper = new FastNanoClockAndSleeper(); BackOff backoff = new AttemptBoundedExponentialBackOff( 5 /* attempts */, 1000 /* initialIntervalMillis */); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - loadService.startLoadJob("jobId", loadConfig, sleeper, backoff); + JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); + verify(response, times(2)).getStatusCode(); verify(response, times(2)).getContent(); verify(response, times(2)).getContentType(); } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#pollJobStatus} succeeds. + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} succeeds. */ @Test - public void testPollJobStatusSucceeds() throws IOException, InterruptedException { + public void testPollJobSucceeds() throws IOException, InterruptedException { Job testJob = new Job(); testJob.setStatus(new JobStatus().setState("DONE")); @@ -183,22 +196,22 @@ public void testPollJobStatusSucceeds() throws IOException, InterruptedException when(response.getStatusCode()).thenReturn(200); when(response.getContent()).thenReturn(toStream(testJob)); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - BigQueryServices.Status status = - loadService.pollJobStatus("projectId", "jobId", Sleeper.DEFAULT, BackOff.ZERO_BACKOFF); + BigQueryServicesImpl.JobServiceImpl jobService = + new BigQueryServicesImpl.JobServiceImpl(bigquery); + Job job = + jobService.pollJob("projectId", "jobId", Sleeper.DEFAULT, BackOff.ZERO_BACKOFF); - assertEquals(BigQueryServices.Status.SUCCEEDED, status); + assertEquals(testJob, job); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); verify(response, times(1)).getContentType(); } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#pollJobStatus} fails. + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} fails. */ @Test - public void testPollJobStatusFailed() throws IOException, InterruptedException { + public void testPollJobFailed() throws IOException, InterruptedException { Job testJob = new Job(); testJob.setStatus(new JobStatus().setState("DONE").setErrorResult(new ErrorProto())); @@ -206,22 +219,22 @@ public void testPollJobStatusFailed() throws IOException, InterruptedException { when(response.getStatusCode()).thenReturn(200); when(response.getContent()).thenReturn(toStream(testJob)); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - BigQueryServices.Status status = - loadService.pollJobStatus("projectId", "jobId", Sleeper.DEFAULT, BackOff.ZERO_BACKOFF); + BigQueryServicesImpl.JobServiceImpl jobService = + new BigQueryServicesImpl.JobServiceImpl(bigquery); + Job job = + jobService.pollJob("projectId", "jobId", Sleeper.DEFAULT, BackOff.ZERO_BACKOFF); - assertEquals(BigQueryServices.Status.FAILED, status); + assertEquals(testJob, job); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); verify(response, times(1)).getContentType(); } /** - * Tests that {@link BigQueryServicesImpl.LoadServiceImpl#pollJobStatus} returns UNKNOWN. + * Tests that {@link BigQueryServicesImpl.JobServiceImpl#pollJob} returns UNKNOWN. */ @Test - public void testPollJobStatusUnknown() throws IOException, InterruptedException { + public void testPollJobUnknown() throws IOException, InterruptedException { Job testJob = new Job(); testJob.setStatus(new JobStatus()); @@ -229,12 +242,12 @@ public void testPollJobStatusUnknown() throws IOException, InterruptedException when(response.getStatusCode()).thenReturn(200); when(response.getContent()).thenReturn(toStream(testJob)); - BigQueryServicesImpl.LoadServiceImpl loadService = - new BigQueryServicesImpl.LoadServiceImpl(bigquery); - BigQueryServices.Status status = - loadService.pollJobStatus("projectId", "jobId", Sleeper.DEFAULT, BackOff.STOP_BACKOFF); + BigQueryServicesImpl.JobServiceImpl jobService = + new BigQueryServicesImpl.JobServiceImpl(bigquery); + Job job = + jobService.pollJob("projectId", "jobId", Sleeper.DEFAULT, BackOff.STOP_BACKOFF); - assertEquals(BigQueryServices.Status.UNKNOWN, status); + assertEquals(null, job); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); verify(response, times(1)).getContentType(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java new file mode 100644 index 000000000000..ecab6f8eb6c5 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowFn; + +import java.util.Collection; + +/** + * A {@link WindowFn} for use during tests that returns the input window for calls to + * {@link #getSideInputWindow(BoundedWindow)}. + */ +class IdentitySideInputWindowFn extends NonMergingWindowFn { + @Override + public Collection assignWindows(WindowFn.AssignContext c) + throws Exception { + return (Collection) c.windows(); + } + + @Override + public boolean isCompatible(WindowFn other) { + return true; + } + + @Override + public Coder windowCoder() { + // not used + return (Coder) GlobalWindow.Coder.INSTANCE; + } + + @Override + public BoundedWindow getSideInputWindow(BoundedWindow window) { + return window; + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubApiaryClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubApiaryClientTest.java new file mode 100644 index 000000000000..40c31fb5ac03 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubApiaryClientTest.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static org.junit.Assert.assertEquals; + +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; + +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PublishResponse; +import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.api.services.pubsub.model.PullRequest; +import com.google.api.services.pubsub.model.PullResponse; +import com.google.api.services.pubsub.model.ReceivedMessage; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.hash.Hashing; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.List; + +/** + * Tests for PubsubApiaryClient. + */ +public class PubsubApiaryClientTest { + private Pubsub mockPubsub; + private PubsubClient client; + + private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic"); + private static final SubscriptionPath SUBSCRIPTION = + PubsubClient.subscriptionPathFromName("testProject", "testSubscription"); + private static final long REQ_TIME = 1234L; + private static final long PUB_TIME = 3456L; + private static final long MESSAGE_TIME = 6789L; + private static final String TIMESTAMP_LABEL = "timestamp"; + private static final String ID_LABEL = "id"; + private static final String MESSAGE_ID = "testMessageId"; + private static final String DATA = "testData"; + private static final String CUSTOM_ID = + Hashing.murmur3_128().hashBytes(DATA.getBytes()).toString(); + private static final String ACK_ID = "testAckId"; + + @Before + public void setup() throws IOException { + mockPubsub = Mockito.mock(Pubsub.class, Mockito.RETURNS_DEEP_STUBS); + client = new PubsubApiaryClient(TIMESTAMP_LABEL, ID_LABEL, mockPubsub); + } + + @After + public void teardown() throws IOException { + client.close(); + client = null; + mockPubsub = null; + } + + @Test + public void pullOneMessage() throws IOException { + String expectedSubscription = SUBSCRIPTION.getPath(); + PullRequest expectedRequest = + new PullRequest().setReturnImmediately(true).setMaxMessages(10); + PubsubMessage expectedPubsubMessage = new PubsubMessage() + .setMessageId(MESSAGE_ID) + .encodeData(DATA.getBytes()) + .setPublishTime(String.valueOf(PUB_TIME)) + .setAttributes( + ImmutableMap.of(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME), + ID_LABEL, CUSTOM_ID)); + ReceivedMessage expectedReceivedMessage = + new ReceivedMessage().setMessage(expectedPubsubMessage) + .setAckId(ACK_ID); + PullResponse expectedResponse = + new PullResponse().setReceivedMessages(ImmutableList.of(expectedReceivedMessage)); + Mockito.when(mockPubsub.projects() + .subscriptions() + .pull(expectedSubscription, expectedRequest) + .execute()) + .thenReturn(expectedResponse); + List acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true); + assertEquals(1, acutalMessages.size()); + IncomingMessage actualMessage = acutalMessages.get(0); + assertEquals(ACK_ID, actualMessage.ackId); + assertEquals(DATA, new String(actualMessage.elementBytes)); + assertEquals(CUSTOM_ID, new String(actualMessage.recordId)); + assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch); + assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch); + } + + @Test + public void publishOneMessage() throws IOException { + String expectedTopic = TOPIC.getPath(); + PubsubMessage expectedPubsubMessage = new PubsubMessage() + .encodeData(DATA.getBytes()) + .setAttributes( + ImmutableMap.of(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME), + ID_LABEL, CUSTOM_ID)); + PublishRequest expectedRequest = new PublishRequest() + .setMessages(ImmutableList.of(expectedPubsubMessage)); + PublishResponse expectedResponse = new PublishResponse() + .setMessageIds(ImmutableList.of(MESSAGE_ID)); + Mockito.when(mockPubsub.projects() + .topics() + .publish(expectedTopic, expectedRequest) + .execute()) + .thenReturn(expectedResponse); + OutgoingMessage actualMessage = new OutgoingMessage(DATA.getBytes(), MESSAGE_TIME); + int n = client.publish(TOPIC, ImmutableList.of(actualMessage)); + assertEquals(1, n); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java new file mode 100644 index 000000000000..22508572e5a0 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static org.junit.Assert.assertEquals; + +import org.apache.beam.sdk.util.PubsubClient.ProjectPath; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; + +import com.google.common.collect.ImmutableMap; + +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Map; + +/** + * Tests for helper classes and methods in PubsubClient. + */ +public class PubsubClientTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + // + // Timestamp handling + // + + private long parse(String timestamp) { + Map map = ImmutableMap.of("myLabel", timestamp); + return PubsubClient.extractTimestamp("myLabel", null, map); + } + + private void roundTripRfc339(String timestamp) { + assertEquals(Instant.parse(timestamp).getMillis(), parse(timestamp)); + } + + private void truncatedRfc339(String timestamp, String truncatedTimestmap) { + assertEquals(Instant.parse(truncatedTimestmap).getMillis(), parse(timestamp)); + } + + @Test + public void noTimestampLabelReturnsPubsubPublish() { + final long time = 987654321L; + long timestamp = PubsubClient.extractTimestamp(null, String.valueOf(time), null); + assertEquals(time, timestamp); + } + + @Test + public void noTimestampLabelAndInvalidPubsubPublishThrowsError() { + thrown.expect(NumberFormatException.class); + PubsubClient.extractTimestamp(null, "not-a-date", null); + } + + @Test + public void timestampLabelWithNullAttributesThrowsError() { + thrown.expect(RuntimeException.class); + thrown.expectMessage("PubSub message is missing a value for timestamp label myLabel"); + PubsubClient.extractTimestamp("myLabel", null, null); + } + + @Test + public void timestampLabelSetWithMissingAttributeThrowsError() { + thrown.expect(RuntimeException.class); + thrown.expectMessage("PubSub message is missing a value for timestamp label myLabel"); + Map map = ImmutableMap.of("otherLabel", "whatever"); + PubsubClient.extractTimestamp("myLabel", null, map); + } + + @Test + public void timestampLabelParsesMillisecondsSinceEpoch() { + long time = 1446162101123L; + Map map = ImmutableMap.of("myLabel", String.valueOf(time)); + long timestamp = PubsubClient.extractTimestamp("myLabel", null, map); + assertEquals(time, timestamp); + } + + @Test + public void timestampLabelParsesRfc3339Seconds() { + roundTripRfc339("2015-10-29T23:41:41Z"); + } + + @Test + public void timestampLabelParsesRfc3339Tenths() { + roundTripRfc339("2015-10-29T23:41:41.1Z"); + } + + @Test + public void timestampLabelParsesRfc3339Hundredths() { + roundTripRfc339("2015-10-29T23:41:41.12Z"); + } + + @Test + public void timestampLabelParsesRfc3339Millis() { + roundTripRfc339("2015-10-29T23:41:41.123Z"); + } + + @Test + public void timestampLabelParsesRfc3339Micros() { + // Note: micros part 456/1000 is dropped. + truncatedRfc339("2015-10-29T23:41:41.123456Z", "2015-10-29T23:41:41.123Z"); + } + + @Test + public void timestampLabelParsesRfc3339MicrosRounding() { + // Note: micros part 999/1000 is dropped, not rounded up. + truncatedRfc339("2015-10-29T23:41:41.123999Z", "2015-10-29T23:41:41.123Z"); + } + + @Test + public void timestampLabelWithInvalidFormatThrowsError() { + thrown.expect(NumberFormatException.class); + parse("not-a-timestamp"); + } + + @Test + public void timestampLabelWithInvalidFormat2ThrowsError() { + thrown.expect(NumberFormatException.class); + parse("null"); + } + + @Test + public void timestampLabelWithInvalidFormat3ThrowsError() { + thrown.expect(NumberFormatException.class); + parse("2015-10"); + } + + @Test + public void timestampLabelParsesRfc3339WithSmallYear() { + // Google and JodaTime agree on dates after 1582-10-15, when the Gregorian Calendar was adopted + // This is therefore a "small year" until this difference is reconciled. + roundTripRfc339("1582-10-15T01:23:45.123Z"); + } + + @Test + public void timestampLabelParsesRfc3339WithLargeYear() { + // Year 9999 in range. + roundTripRfc339("9999-10-29T23:41:41.123999Z"); + } + + @Test + public void timestampLabelRfc3339WithTooLargeYearThrowsError() { + thrown.expect(NumberFormatException.class); + // Year 10000 out of range. + parse("10000-10-29T23:41:41.123999Z"); + } + + // + // Paths + // + + @Test + public void projectPathFromIdWellFormed() { + ProjectPath path = PubsubClient.projectPathFromId("test"); + assertEquals("projects/test", path.getPath()); + } + + @Test + public void subscriptionPathFromNameWellFormed() { + SubscriptionPath path = PubsubClient.subscriptionPathFromName("test", "something"); + assertEquals("projects/test/subscriptions/something", path.getPath()); + assertEquals("/subscriptions/test/something", path.getV1Beta1Path()); + } + + @Test + public void topicPathFromNameWellFormed() { + TopicPath path = PubsubClient.topicPathFromName("test", "something"); + assertEquals("projects/test/topics/something", path.getPath()); + assertEquals("/topics/test/something", path.getV1Beta1Path()); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java new file mode 100644 index 000000000000..189049c07ea4 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static org.junit.Assert.assertEquals; + +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.hash.Hashing; +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; +import com.google.pubsub.v1.PublishRequest; +import com.google.pubsub.v1.PublishResponse; +import com.google.pubsub.v1.PublisherGrpc; +import com.google.pubsub.v1.PubsubMessage; +import com.google.pubsub.v1.PullRequest; +import com.google.pubsub.v1.PullResponse; +import com.google.pubsub.v1.ReceivedMessage; +import com.google.pubsub.v1.SubscriberGrpc; + +import io.grpc.ManagedChannel; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.List; + +/** + * Tests for PubsubGrpcClient. + */ +public class PubsubGrpcClientTest { + private ManagedChannel mockChannel; + private GoogleCredentials mockCredentials; + private PublisherGrpc.PublisherBlockingStub mockPublisherStub; + private SubscriberGrpc.SubscriberBlockingStub mockSubscriberStub; + + private PubsubClient client; + + private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic"); + private static final SubscriptionPath SUBSCRIPTION = + PubsubClient.subscriptionPathFromName("testProject", "testSubscription"); + private static final long REQ_TIME = 1234L; + private static final long PUB_TIME = 3456L; + private static final long MESSAGE_TIME = 6789L; + private static final String TIMESTAMP_LABEL = "timestamp"; + private static final String ID_LABEL = "id"; + private static final String MESSAGE_ID = "testMessageId"; + private static final String DATA = "testData"; + private static final String CUSTOM_ID = + Hashing.murmur3_128().hashBytes(DATA.getBytes()).toString(); + private static final String ACK_ID = "testAckId"; + + @Before + public void setup() throws IOException { + mockChannel = Mockito.mock(ManagedChannel.class); + mockCredentials = Mockito.mock(GoogleCredentials.class); + mockPublisherStub = + Mockito.mock(PublisherGrpc.PublisherBlockingStub.class, Mockito.RETURNS_DEEP_STUBS); + mockSubscriberStub = + Mockito.mock(SubscriberGrpc.SubscriberBlockingStub.class, Mockito.RETURNS_DEEP_STUBS); + client = new PubsubGrpcClient(TIMESTAMP_LABEL, ID_LABEL, 0, mockChannel, + mockCredentials, mockPublisherStub, mockSubscriberStub); + } + + @After + public void teardown() throws IOException { + client.close(); + client = null; + mockChannel = null; + mockCredentials = null; + mockPublisherStub = null; + mockSubscriberStub = null; + } + + @Test + public void pullOneMessage() throws IOException { + String expectedSubscription = SUBSCRIPTION.getPath(); + PullRequest expectedRequest = + PullRequest.newBuilder() + .setSubscription(expectedSubscription) + .setReturnImmediately(true) + .setMaxMessages(10) + .build(); + Timestamp timestamp = Timestamp.newBuilder() + .setSeconds(PUB_TIME / 1000) + .setNanos((int) (PUB_TIME % 1000) * 1000) + .build(); + PubsubMessage expectedPubsubMessage = + PubsubMessage.newBuilder() + .setMessageId(MESSAGE_ID) + .setData( + ByteString.copyFrom(DATA.getBytes())) + .setPublishTime(timestamp) + .putAllAttributes( + ImmutableMap.of(TIMESTAMP_LABEL, + String.valueOf(MESSAGE_TIME), + ID_LABEL, CUSTOM_ID)) + .build(); + ReceivedMessage expectedReceivedMessage = + ReceivedMessage.newBuilder() + .setMessage(expectedPubsubMessage) + .setAckId(ACK_ID) + .build(); + PullResponse expectedResponse = + PullResponse.newBuilder() + .addAllReceivedMessages(ImmutableList.of(expectedReceivedMessage)) + .build(); + Mockito.when(mockSubscriberStub.pull(expectedRequest)) + .thenReturn(expectedResponse); + List acutalMessages = client.pull(REQ_TIME, SUBSCRIPTION, 10, true); + assertEquals(1, acutalMessages.size()); + IncomingMessage actualMessage = acutalMessages.get(0); + assertEquals(ACK_ID, actualMessage.ackId); + assertEquals(DATA, new String(actualMessage.elementBytes)); + assertEquals(CUSTOM_ID, new String(actualMessage.recordId)); + assertEquals(REQ_TIME, actualMessage.requestTimeMsSinceEpoch); + assertEquals(MESSAGE_TIME, actualMessage.timestampMsSinceEpoch); + } + + @Test + public void publishOneMessage() throws IOException { + String expectedTopic = TOPIC.getPath(); + PubsubMessage expectedPubsubMessage = + PubsubMessage.newBuilder() + .setData(ByteString.copyFrom(DATA.getBytes())) + .putAllAttributes( + ImmutableMap.of(TIMESTAMP_LABEL, String.valueOf(MESSAGE_TIME), + ID_LABEL, CUSTOM_ID)) + .build(); + PublishRequest expectedRequest = + PublishRequest.newBuilder() + .setTopic(expectedTopic) + .addAllMessages( + ImmutableList.of(expectedPubsubMessage)) + .build(); + PublishResponse expectedResponse = + PublishResponse.newBuilder() + .addAllMessageIds(ImmutableList.of(MESSAGE_ID)) + .build(); + Mockito.when(mockPublisherStub.publish(expectedRequest)) + .thenReturn(expectedResponse); + OutgoingMessage actualMessage = new OutgoingMessage(DATA.getBytes(), MESSAGE_TIME); + int n = client.publish(TOPIC, ImmutableList.of(actualMessage)); + assertEquals(1, n); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java new file mode 100644 index 000000000000..7d8513b725b1 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.util; + +import static org.junit.Assert.assertEquals; + +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import org.junit.Test; + +import java.io.IOException; +import java.util.List; + +/** + * Tests for PubsubTestClient. + */ +public class PubsubTestClientTest { + private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic"); + private static final SubscriptionPath SUBSCRIPTION = + PubsubClient.subscriptionPathFromName("testProject", "testSubscription"); + private static final long REQ_TIME = 1234L; + private static final long MESSAGE_TIME = 6789L; + private static final String MESSAGE_ID = "testMessageId"; + private static final String DATA = "testData"; + private static final String ACK_ID = "testAckId"; + private static final int ACK_TIMEOUT_S = 60; + + @Test + public void pullOneMessage() throws IOException { + IncomingMessage expectedIncomingMessage = + new IncomingMessage(DATA.getBytes(), MESSAGE_TIME, REQ_TIME, ACK_ID, MESSAGE_ID.getBytes()); + try (PubsubTestClient client = + new PubsubTestClient(null, SUBSCRIPTION, ACK_TIMEOUT_S, null, + Lists.newArrayList(expectedIncomingMessage))) { + long now = REQ_TIME; + client.advanceTo(now); + List incomingMessages = client.pull(now, SUBSCRIPTION, 1, true); + assertEquals(1, incomingMessages.size()); + assertEquals(expectedIncomingMessage, incomingMessages.get(0)); + // Timeout on ACK. + now += (ACK_TIMEOUT_S + 10) * 1000; + client.advanceTo(now); + incomingMessages = client.pull(now, SUBSCRIPTION, 1, true); + assertEquals(1, incomingMessages.size()); + assertEquals(expectedIncomingMessage.withRequestTime(now), incomingMessages.get(0)); + now += 10 * 1000; + client.advanceTo(now); + // Extend ack + client.modifyAckDeadline(SUBSCRIPTION, ImmutableList.of(ACK_ID), 20); + // Timeout on extended ACK + now += 30 * 1000; + client.advanceTo(now); + incomingMessages = client.pull(now, SUBSCRIPTION, 1, true); + assertEquals(1, incomingMessages.size()); + assertEquals(expectedIncomingMessage.withRequestTime(now), incomingMessages.get(0)); + // Extend ack + client.modifyAckDeadline(SUBSCRIPTION, ImmutableList.of(ACK_ID), 20); + // Ack + now += 15 * 1000; + client.advanceTo(now); + client.acknowledge(SUBSCRIPTION, ImmutableList.of(ACK_ID)); + } + } + + @Test + public void publishOneMessage() throws IOException { + OutgoingMessage expectedOutgoingMessage = new OutgoingMessage(DATA.getBytes(), MESSAGE_TIME); + try (PubsubTestClient client = + new PubsubTestClient(TOPIC, null, ACK_TIMEOUT_S, + Sets.newHashSet(expectedOutgoingMessage), null)) { + client.publish(TOPIC, ImmutableList.of(expectedOutgoingMessage)); + } + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java new file mode 100644 index 000000000000..88851187bb65 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.util; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.emptyIterable; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.when; + +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; + +import com.google.common.collect.ImmutableList; + +import org.hamcrest.Matchers; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; + +import java.util.ArrayList; +import java.util.List; + +/** + * Tests for {@link PushbackSideInputDoFnRunner}. + */ +@RunWith(JUnit4.class) +public class PushbackSideInputDoFnRunnerTest { + @Mock private ReadyCheckingSideInputReader reader; + private TestDoFnRunner underlying; + private PCollectionView singletonView; + + @Before + public void setup() { + MockitoAnnotations.initMocks(this); + TestPipeline p = TestPipeline.create(); + PCollection created = p.apply(Create.of(1, 2, 3)); + singletonView = + created + .apply(Window.into(new IdentitySideInputWindowFn())) + .apply(Sum.integersGlobally().asSingletonView()); + + underlying = new TestDoFnRunner<>(); + } + + private PushbackSideInputDoFnRunner createRunner( + ImmutableList> views) { + PushbackSideInputDoFnRunner runner = + PushbackSideInputDoFnRunner.create(underlying, views, reader); + runner.startBundle(); + return runner; + } + + @Test + public void startFinishBundleDelegates() { + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of(singletonView)); + + assertThat(underlying.started, is(true)); + assertThat(underlying.finished, is(false)); + runner.finishBundle(); + assertThat(underlying.finished, is(true)); + } + + @Test + public void processElementSideInputNotReady() { + when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class))) + .thenReturn(false); + + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of(singletonView)); + + WindowedValue oneWindow = + WindowedValue.of( + 2, + new Instant(-2), + new IntervalWindow(new Instant(-500L), new Instant(0L)), + PaneInfo.ON_TIME_AND_ONLY_FIRING); + Iterable> oneWindowPushback = + runner.processElementInReadyWindows(oneWindow); + assertThat(oneWindowPushback, containsInAnyOrder(oneWindow)); + assertThat(underlying.inputElems, Matchers.>emptyIterable()); + } + + @Test + public void processElementSideInputNotReadyMultipleWindows() { + when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class))) + .thenReturn(false); + + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of(singletonView)); + + WindowedValue multiWindow = + WindowedValue.of( + 2, + new Instant(-2), + ImmutableList.of( + new IntervalWindow(new Instant(-500L), new Instant(0L)), + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)), + GlobalWindow.INSTANCE), + PaneInfo.ON_TIME_AND_ONLY_FIRING); + Iterable> multiWindowPushback = + runner.processElementInReadyWindows(multiWindow); + assertThat(multiWindowPushback, equalTo(multiWindow.explodeWindows())); + assertThat(underlying.inputElems, Matchers.>emptyIterable()); + } + + @Test + public void processElementSideInputNotReadySomeWindows() { + when(reader.isReady(Mockito.eq(singletonView), Mockito.eq(GlobalWindow.INSTANCE))) + .thenReturn(false); + when( + reader.isReady( + Mockito.eq(singletonView), + org.mockito.AdditionalMatchers.not(Mockito.eq(GlobalWindow.INSTANCE)))) + .thenReturn(true); + + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of(singletonView)); + + IntervalWindow littleWindow = new IntervalWindow(new Instant(-500L), new Instant(0L)); + IntervalWindow bigWindow = + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)); + WindowedValue multiWindow = + WindowedValue.of( + 2, + new Instant(-2), + ImmutableList.of(littleWindow, bigWindow, GlobalWindow.INSTANCE), + PaneInfo.NO_FIRING); + Iterable> multiWindowPushback = + runner.processElementInReadyWindows(multiWindow); + assertThat( + multiWindowPushback, + containsInAnyOrder(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-2L)))); + assertThat(underlying.inputElems, + containsInAnyOrder(WindowedValue.of(2, new Instant(-2), littleWindow, PaneInfo.NO_FIRING), + WindowedValue.of(2, new Instant(-2), bigWindow, PaneInfo.NO_FIRING))); + } + + @Test + public void processElementSideInputReadyAllWindows() { + when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class))) + .thenReturn(true); + + ImmutableList> views = ImmutableList.>of(singletonView); + PushbackSideInputDoFnRunner runner = createRunner(views); + + WindowedValue multiWindow = + WindowedValue.of( + 2, + new Instant(-2), + ImmutableList.of( + new IntervalWindow(new Instant(-500L), new Instant(0L)), + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)), + GlobalWindow.INSTANCE), + PaneInfo.ON_TIME_AND_ONLY_FIRING); + Iterable> multiWindowPushback = + runner.processElementInReadyWindows(multiWindow); + assertThat(multiWindowPushback, emptyIterable()); + assertThat(underlying.inputElems, + containsInAnyOrder(ImmutableList.copyOf(multiWindow.explodeWindows()).toArray())); + } + + @Test + public void processElementNoSideInputs() { + PushbackSideInputDoFnRunner runner = + createRunner(ImmutableList.>of()); + + WindowedValue multiWindow = + WindowedValue.of( + 2, + new Instant(-2), + ImmutableList.of( + new IntervalWindow(new Instant(-500L), new Instant(0L)), + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)), + GlobalWindow.INSTANCE), + PaneInfo.ON_TIME_AND_ONLY_FIRING); + Iterable> multiWindowPushback = + runner.processElementInReadyWindows(multiWindow); + assertThat(multiWindowPushback, emptyIterable()); + assertThat(underlying.inputElems, containsInAnyOrder(multiWindow)); + } + + private static class TestDoFnRunner implements DoFnRunner { + List> inputElems; + private boolean started = false; + private boolean finished = false; + + @Override + public void startBundle() { + started = true; + inputElems = new ArrayList<>(); + } + + @Override + public void processElement(WindowedValue elem) { + inputElems.add(elem); + } + + @Override + public void finishBundle() { + finished = true; + } + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java index f0d2a44a1ae9..f296d653074b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java @@ -58,7 +58,6 @@ import com.google.common.base.Function; import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -408,7 +407,7 @@ public WindowedValue apply(TimestampedValue input) { windowFn, value, timestamp, Arrays.asList(GlobalWindow.INSTANCE))); return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING); } catch (Exception e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } } })); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java index 10d3d3521ca5..0889b4f66ca7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; - import static org.junit.Assert.assertTrue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -45,7 +44,6 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -257,7 +255,7 @@ public final void injectElements(Collection> values) th windowedValues.add(WindowedValue.of(value, timestamp, assignedWindows, PaneInfo.NO_FIRING)); } catch (Exception e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } } diff --git a/sdks/java/extensions/join-library/pom.xml b/sdks/java/extensions/join-library/pom.xml index e5ae1e270f65..64b3bfc25877 100644 --- a/sdks/java/extensions/join-library/pom.xml +++ b/sdks/java/extensions/join-library/pom.xml @@ -36,30 +36,18 @@ org.apache.maven.plugins maven-compiler-plugin - 3.2 - - 1.7 - 1.7 - - org.apache.maven.plugins maven-checkstyle-plugin - org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - - jar - - - + + + org.apache.maven.plugins + maven-jar-plugin @@ -68,27 +56,23 @@ org.apache.beam java-sdk-all - ${project.version} com.google.guava guava - 19.0 org.hamcrest hamcrest-all - 1.3 test junit junit - 4.12 test diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index d1d5cd674895..18e46834b243 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -37,17 +37,36 @@ 0.2.3 + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + + org.apache.beam java-sdk-all - ${project.version} io.grpc grpc-all - 0.12.0 @@ -66,7 +85,6 @@ org.apache.beam java-sdk-all - ${project.version} tests test @@ -74,21 +92,18 @@ org.hamcrest hamcrest-all - 1.3 test junit junit - 4.11 test org.slf4j slf4j-jdk14 - 1.7.14 test diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml index 3eaef628b556..e2d7eb2d1ec7 100644 --- a/sdks/java/io/hdfs/pom.xml +++ b/sdks/java/io/hdfs/pom.xml @@ -31,13 +31,32 @@ hdfs Apache Beam :: SDKs :: Java :: IO :: HDFS Library to read and write Hadoop/HDFS file formats from Beam. - jar + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.apache.beam java-sdk-all - ${project.version} @@ -51,14 +70,12 @@ org.hamcrest hamcrest-all - 1.3 test junit junit - 4.11 test diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml index ddfceed8ddb1..bb9baaa1f178 100644 --- a/sdks/java/io/kafka/pom.xml +++ b/sdks/java/io/kafka/pom.xml @@ -30,27 +30,32 @@ kafka Apache Beam :: SDKs :: Java :: IO :: Kafka Library to read Kafka topics. - jar - - - org.apache.maven.plugins - maven-compiler-plugin - - - - org.apache.maven.plugins - maven-checkstyle-plugin - - + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + org.apache.beam java-sdk-all - ${project.version} @@ -63,21 +68,18 @@ org.hamcrest hamcrest-all - ${hamcrest.version} test junit junit - ${junit.version} test org.slf4j slf4j-jdk14 - ${slf4j.version} test diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index e6053116fbe5..4f353df865ff 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -44,7 +44,6 @@ import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; -import com.google.common.base.Throwables; import com.google.common.collect.ComparisonChain; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -634,7 +633,7 @@ public UnboundedKafkaReader createReader(PipelineOptions options, return new UnboundedKafkaReader( generateInitialSplits(1, options).get(0), checkpointMark); } catch (Exception e) { - Throwables.propagate(e); + throw new RuntimeException(e); } } return new UnboundedKafkaReader(this, checkpointMark); diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml index 27c40f7670b0..1032ae9245bf 100644 --- a/sdks/java/java8tests/pom.xml +++ b/sdks/java/java8tests/pom.xml @@ -35,8 +35,6 @@ This artifact includes tests of the SDK from a Java 8 user. - jar - @@ -77,42 +75,11 @@ org.apache.maven.plugins maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - org.apache.maven.plugins maven-jar-plugin - - - default-jar - - jar - - - - default-test-jar - - test-jar - - - @@ -127,35 +94,30 @@ org.apache.beam java-sdk-all - ${project.version} test com.google.guava guava - ${guava.version} test joda-time joda-time - ${joda.version} test org.hamcrest hamcrest-all - ${hamcrest.version} test junit junit - ${junit.version} test diff --git a/sdks/pom.xml b/sdks/pom.xml index 808f0d33c8c5..0567f5756eee 100644 --- a/sdks/pom.xml +++ b/sdks/pom.xml @@ -38,4 +38,25 @@ java + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + default-test-jar + + test-jar + + + + + + + + \ No newline at end of file