You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by dh...@apache.org on 2016/04/27 03:08:33 UTC
[01/21] incubator-beam git commit: Update Dataflow worker harness
container image to match package changes in this pull request
Repository: incubator-beam
Updated Branches:
refs/heads/master 5e3d7ad20 -> e3105c8e1
Update Dataflow worker harness container image to match package changes
in this pull request
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/0fafd4e8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/0fafd4e8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/0fafd4e8
Branch: refs/heads/master
Commit: 0fafd4e89f387b73c33aac65d42a6a367e9dd738
Parents: 0219098
Author: Davor Bonaci <da...@google.com>
Authored: Tue Apr 26 16:38:06 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Apr 26 17:59:39 2016 -0700
----------------------------------------------------------------------
.../org/apache/beam/runners/dataflow/DataflowPipelineRunner.java | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fafd4e8/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java
index 0fc095a..ec4a60c 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java
@@ -217,9 +217,9 @@ public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob>
// Default Docker container images that execute Dataflow worker harness, residing in Google
// Container Registry, separately for Batch and Streaming.
public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE
- = "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160422";
+ = "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160426";
public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE
- = "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160422";
+ = "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160426";
// The limit of CreateJob request size.
private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024;
[04/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java
deleted file mode 100644
index f888c5b..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerTest.java
+++ /dev/null
@@ -1,1400 +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.runners;
-
-import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasItem;
-import static org.hamcrest.Matchers.instanceOf;
-import static org.hamcrest.Matchers.startsWith;
-import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.AvroSource;
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.DataflowPipelineDebugOptions;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner.BatchViewAsList;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner.BatchViewAsMap;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner.BatchViewAsMultimap;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner.TransformedMap;
-import org.apache.beam.sdk.runners.dataflow.TestCountingSource;
-import org.apache.beam.sdk.runners.worker.IsmFormat;
-import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecord;
-import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecordCoder;
-import org.apache.beam.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFnTester;
-import org.apache.beam.sdk.transforms.PTransform;
-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.util.CoderUtils;
-import org.apache.beam.sdk.util.GcsUtil;
-import org.apache.beam.sdk.util.NoopPathValidator;
-import org.apache.beam.sdk.util.ReleaseInfo;
-import org.apache.beam.sdk.util.TestCredential;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.PValue;
-import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.ListJobsResponse;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-import org.hamcrest.Description;
-import org.hamcrest.Matchers;
-import org.hamcrest.TypeSafeMatcher;
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.internal.matchers.ThrowableMessageMatcher;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SeekableByteChannel;
-import java.nio.file.Files;
-import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Pattern;
-
-/**
- * Tests for the {@link DataflowPipelineRunner}.
- */
-@RunWith(JUnit4.class)
-public class DataflowPipelineRunnerTest {
-
- private static final String PROJECT_ID = "some-project";
-
- @Rule
- public TemporaryFolder tmpFolder = new TemporaryFolder();
- @Rule
- public ExpectedException thrown = ExpectedException.none();
-
- // Asserts that the given Job has all expected fields set.
- private static void assertValidJob(Job job) {
- assertNull(job.getId());
- assertNull(job.getCurrentState());
- assertTrue(Pattern.matches("[a-z]([-a-z0-9]*[a-z0-9])?", job.getName()));
- }
-
- private Pipeline buildDataflowPipeline(DataflowPipelineOptions options) {
- options.setStableUniqueNames(CheckEnabled.ERROR);
- options.setRunner(DataflowPipelineRunner.class);
- Pipeline p = Pipeline.create(options);
-
- p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
- .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
-
- return p;
- }
-
- private static Dataflow buildMockDataflow(
- final ArgumentCaptor<Job> jobCaptor) throws IOException {
- Dataflow mockDataflowClient = mock(Dataflow.class);
- Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
- Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
- Dataflow.Projects.Jobs.Create mockRequest =
- mock(Dataflow.Projects.Jobs.Create.class);
- Dataflow.Projects.Jobs.List mockList = mock(Dataflow.Projects.Jobs.List.class);
-
- when(mockDataflowClient.projects()).thenReturn(mockProjects);
- when(mockProjects.jobs()).thenReturn(mockJobs);
- when(mockJobs.create(eq(PROJECT_ID), jobCaptor.capture()))
- .thenReturn(mockRequest);
- when(mockJobs.list(eq(PROJECT_ID))).thenReturn(mockList);
- when(mockList.setPageToken(anyString())).thenReturn(mockList);
- when(mockList.execute())
- .thenReturn(
- new ListJobsResponse()
- .setJobs(
- Arrays.asList(
- new Job()
- .setName("oldjobname")
- .setId("oldJobId")
- .setCurrentState("JOB_STATE_RUNNING"))));
-
- Job resultJob = new Job();
- resultJob.setId("newid");
- when(mockRequest.execute()).thenReturn(resultJob);
- return mockDataflowClient;
- }
-
- private GcsUtil buildMockGcsUtil(boolean bucketExists) throws IOException {
- GcsUtil mockGcsUtil = mock(GcsUtil.class);
- when(mockGcsUtil.create(any(GcsPath.class), anyString()))
- .then(new Answer<SeekableByteChannel>() {
- @Override
- public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
- return FileChannel.open(
- Files.createTempFile("channel-", ".tmp"),
- StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
- }
- });
-
- when(mockGcsUtil.isGcsPatternSupported(anyString())).thenReturn(true);
- when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer<List<GcsPath>>() {
- @Override
- public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
- return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
- }
- });
- when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(bucketExists);
- return mockGcsUtil;
- }
-
- private DataflowPipelineOptions buildPipelineOptions() throws IOException {
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
- return buildPipelineOptions(jobCaptor);
- }
-
- private DataflowPipelineOptions buildPipelineOptions(
- ArgumentCaptor<Job> jobCaptor) throws IOException {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setProject(PROJECT_ID);
- options.setTempLocation("gs://somebucket/some/path");
- // Set FILES_PROPERTY to empty to prevent a default value calculated from classpath.
- options.setFilesToStage(new LinkedList<String>());
- options.setDataflowClient(buildMockDataflow(jobCaptor));
- options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
- options.setGcpCredential(new TestCredential());
- return options;
- }
-
- @Test
- public void testFromOptionsWithUppercaseConvertsToLowercase() throws Exception {
- String mixedCase = "ThisJobNameHasMixedCase";
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
- DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
- options.setJobName(mixedCase);
-
- DataflowPipelineRunner runner = DataflowPipelineRunner.fromOptions(options);
- assertThat(options.getJobName(), equalTo(mixedCase.toLowerCase()));
- }
-
- @Test
- public void testRun() throws IOException {
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
- DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
- Pipeline p = buildDataflowPipeline(options);
- DataflowPipelineJob job = (DataflowPipelineJob) p.run();
- assertEquals("newid", job.getJobId());
- assertValidJob(jobCaptor.getValue());
- }
-
- @Test
- public void testRunReturnDifferentRequestId() throws IOException {
- DataflowPipelineOptions options = buildPipelineOptions();
- Dataflow mockDataflowClient = options.getDataflowClient();
- Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
- when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
- .thenReturn(mockRequest);
- Job resultJob = new Job();
- resultJob.setId("newid");
- // Return a different request id.
- resultJob.setClientRequestId("different_request_id");
- when(mockRequest.execute()).thenReturn(resultJob);
-
- Pipeline p = buildDataflowPipeline(options);
- try {
- p.run();
- fail("Expected DataflowJobAlreadyExistsException");
- } catch (DataflowJobAlreadyExistsException expected) {
- assertThat(expected.getMessage(),
- containsString("If you want to submit a second job, try again by setting a "
- + "different name using --jobName."));
- assertEquals(expected.getJob().getJobId(), resultJob.getId());
- }
- }
-
- @Test
- public void testUpdate() throws IOException {
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
- DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
- options.setUpdate(true);
- options.setJobName("oldJobName");
- Pipeline p = buildDataflowPipeline(options);
- DataflowPipelineJob job = (DataflowPipelineJob) p.run();
- assertEquals("newid", job.getJobId());
- assertValidJob(jobCaptor.getValue());
- }
-
- @Test
- public void testUpdateNonExistentPipeline() throws IOException {
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("Could not find running job named badjobname");
-
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setUpdate(true);
- options.setJobName("badJobName");
- Pipeline p = buildDataflowPipeline(options);
- p.run();
- }
-
- @Test
- public void testUpdateAlreadyUpdatedPipeline() throws IOException {
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setUpdate(true);
- options.setJobName("oldJobName");
- Dataflow mockDataflowClient = options.getDataflowClient();
- Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
- when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
- .thenReturn(mockRequest);
- final Job resultJob = new Job();
- resultJob.setId("newid");
- // Return a different request id.
- resultJob.setClientRequestId("different_request_id");
- when(mockRequest.execute()).thenReturn(resultJob);
-
- Pipeline p = buildDataflowPipeline(options);
-
- thrown.expect(DataflowJobAlreadyUpdatedException.class);
- thrown.expect(new TypeSafeMatcher<DataflowJobAlreadyUpdatedException>() {
- @Override
- public void describeTo(Description description) {
- description.appendText("Expected job ID: " + resultJob.getId());
- }
-
- @Override
- protected boolean matchesSafely(DataflowJobAlreadyUpdatedException item) {
- return resultJob.getId().equals(item.getJob().getJobId());
- }
- });
- thrown.expectMessage("The job named oldjobname with id: oldJobId has already been updated "
- + "into job id: newid and cannot be updated again.");
- p.run();
- }
-
- @Test
- public void testRunWithFiles() throws IOException {
- // Test that the function DataflowPipelineRunner.stageFiles works as
- // expected.
- GcsUtil mockGcsUtil = buildMockGcsUtil(true /* bucket exists */);
- final String gcsStaging = "gs://somebucket/some/path";
- final String gcsTemp = "gs://somebucket/some/temp/path";
- final String cloudDataflowDataset = "somedataset";
-
- // Create some temporary files.
- File temp1 = File.createTempFile("DataflowPipelineRunnerTest", "txt");
- temp1.deleteOnExit();
- File temp2 = File.createTempFile("DataflowPipelineRunnerTest2", "txt");
- temp2.deleteOnExit();
-
- String overridePackageName = "alias.txt";
-
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setFilesToStage(ImmutableList.of(
- temp1.getAbsolutePath(),
- overridePackageName + "=" + temp2.getAbsolutePath()));
- options.setStagingLocation(gcsStaging);
- options.setTempLocation(gcsTemp);
- options.setTempDatasetId(cloudDataflowDataset);
- options.setProject(PROJECT_ID);
- options.setJobName("job");
- options.setDataflowClient(buildMockDataflow(jobCaptor));
- options.setGcsUtil(mockGcsUtil);
- options.setGcpCredential(new TestCredential());
-
- Pipeline p = buildDataflowPipeline(options);
-
- DataflowPipelineJob job = (DataflowPipelineJob) p.run();
- assertEquals("newid", job.getJobId());
-
- Job workflowJob = jobCaptor.getValue();
- assertValidJob(workflowJob);
-
- assertEquals(
- 2,
- workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().size());
- DataflowPackage workflowPackage1 =
- workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0);
- assertThat(workflowPackage1.getName(), startsWith(temp1.getName()));
- DataflowPackage workflowPackage2 =
- workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1);
- assertEquals(overridePackageName, workflowPackage2.getName());
-
- assertEquals(
- "storage.googleapis.com/somebucket/some/temp/path",
- workflowJob.getEnvironment().getTempStoragePrefix());
- assertEquals(
- cloudDataflowDataset,
- workflowJob.getEnvironment().getDataset());
- assertEquals(
- ReleaseInfo.getReleaseInfo().getName(),
- workflowJob.getEnvironment().getUserAgent().get("name"));
- assertEquals(
- ReleaseInfo.getReleaseInfo().getVersion(),
- workflowJob.getEnvironment().getUserAgent().get("version"));
- }
-
- @Test
- public void runWithDefaultFilesToStage() throws Exception {
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setFilesToStage(null);
- DataflowPipelineRunner.fromOptions(options);
- assertTrue(!options.getFilesToStage().isEmpty());
- }
-
- @Test
- public void detectClassPathResourceWithFileResources() throws Exception {
- File file = tmpFolder.newFile("file");
- File file2 = tmpFolder.newFile("file2");
- URLClassLoader classLoader = new URLClassLoader(new URL[]{
- file.toURI().toURL(),
- file2.toURI().toURL()
- });
-
- assertEquals(ImmutableList.of(file.getAbsolutePath(), file2.getAbsolutePath()),
- DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader));
- }
-
- @Test
- public void detectClassPathResourcesWithUnsupportedClassLoader() {
- ClassLoader mockClassLoader = Mockito.mock(ClassLoader.class);
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("Unable to use ClassLoader to detect classpath elements.");
-
- DataflowPipelineRunner.detectClassPathResourcesToStage(mockClassLoader);
- }
-
- @Test
- public void detectClassPathResourceWithNonFileResources() throws Exception {
- String url = "http://www.google.com/all-the-secrets.jar";
- URLClassLoader classLoader = new URLClassLoader(new URL[]{
- new URL(url)
- });
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("Unable to convert url (" + url + ") to file.");
-
- DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader);
- }
-
- @Test
- public void testGcsStagingLocationInitialization() throws Exception {
- // Test that the staging location is initialized correctly.
- String gcsTemp = "gs://somebucket/some/temp/path";
-
- // Set temp location (required), and check that staging location is set.
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setTempLocation(gcsTemp);
- options.setProject(PROJECT_ID);
- options.setGcpCredential(new TestCredential());
- options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
- DataflowPipelineRunner.fromOptions(options);
-
- assertNotNull(options.getStagingLocation());
- }
-
- @Test
- public void testNonGcsFilePathInReadFailure() throws IOException {
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
- Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
- p.apply(TextIO.Read.named("ReadMyNonGcsFile").from(tmpFolder.newFile().getPath()));
-
- thrown.expectCause(Matchers.allOf(
- instanceOf(IllegalArgumentException.class),
- ThrowableMessageMatcher.hasMessage(
- containsString("expected a valid 'gs://' path but was given"))));
- p.run();
- assertValidJob(jobCaptor.getValue());
- }
-
- @Test
- public void testNonGcsFilePathInWriteFailure() throws IOException {
- Pipeline p = buildDataflowPipeline(buildPipelineOptions());
- PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
- pc.apply(TextIO.Write.named("WriteMyNonGcsFile").to("/tmp/file"));
- }
-
- @Test
- public void testMultiSlashGcsFileReadPath() throws IOException {
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
- Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
- p.apply(TextIO.Read.named("ReadInvalidGcsFile")
- .from("gs://bucket/tmp//file"));
-
- thrown.expectCause(Matchers.allOf(
- instanceOf(IllegalArgumentException.class),
- ThrowableMessageMatcher.hasMessage(containsString("consecutive slashes"))));
- p.run();
- assertValidJob(jobCaptor.getValue());
- }
-
- @Test
- public void testMultiSlashGcsFileWritePath() throws IOException {
- Pipeline p = buildDataflowPipeline(buildPipelineOptions());
- PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("consecutive slashes");
- pc.apply(TextIO.Write.named("WriteInvalidGcsFile").to("gs://bucket/tmp//file"));
- }
-
- @Test
- public void testInvalidTempLocation() throws IOException {
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
- DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
- options.setTempLocation("file://temp/location");
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
- DataflowPipelineRunner.fromOptions(options);
- assertValidJob(jobCaptor.getValue());
- }
-
- @Test
- public void testInvalidStagingLocation() throws IOException {
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setStagingLocation("file://my/staging/location");
- try {
- DataflowPipelineRunner.fromOptions(options);
- fail("fromOptions should have failed");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
- }
- options.setStagingLocation("my/staging/location");
- try {
- DataflowPipelineRunner.fromOptions(options);
- fail("fromOptions should have failed");
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
- }
- }
-
- @Test
- public void testNonExistentTempLocation() throws IOException {
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
- GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
- DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
- options.setGcsUtil(mockGcsUtil);
- options.setTempLocation("gs://non-existent-bucket/location");
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage(containsString(
- "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
- DataflowPipelineRunner.fromOptions(options);
- assertValidJob(jobCaptor.getValue());
- }
-
- @Test
- public void testNonExistentStagingLocation() throws IOException {
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
- GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
- DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
- options.setGcsUtil(mockGcsUtil);
- options.setStagingLocation("gs://non-existent-bucket/location");
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage(containsString(
- "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
- DataflowPipelineRunner.fromOptions(options);
- assertValidJob(jobCaptor.getValue());
- }
-
- @Test
- public void testNoProjectFails() {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
-
- options.setRunner(DataflowPipelineRunner.class);
- // Explicitly set to null to prevent the default instance factory from reading credentials
- // from a user's environment, causing this test to fail.
- options.setProject(null);
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("Project id");
- thrown.expectMessage("when running a Dataflow in the cloud");
-
- DataflowPipelineRunner.fromOptions(options);
- }
-
- @Test
- public void testProjectId() throws IOException {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setProject("foo-12345");
-
- options.setStagingLocation("gs://spam/ham/eggs");
- options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
- options.setGcpCredential(new TestCredential());
-
- DataflowPipelineRunner.fromOptions(options);
- }
-
- @Test
- public void testProjectPrefix() throws IOException {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setProject("google.com:some-project-12345");
-
- options.setStagingLocation("gs://spam/ham/eggs");
- options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
- options.setGcpCredential(new TestCredential());
-
- DataflowPipelineRunner.fromOptions(options);
- }
-
- @Test
- public void testProjectNumber() throws IOException {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setProject("12345");
-
- options.setStagingLocation("gs://spam/ham/eggs");
- options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("Project ID");
- thrown.expectMessage("project number");
-
- DataflowPipelineRunner.fromOptions(options);
- }
-
- @Test
- public void testProjectDescription() throws IOException {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setProject("some project");
-
- options.setStagingLocation("gs://spam/ham/eggs");
- options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("Project ID");
- thrown.expectMessage("project description");
-
- DataflowPipelineRunner.fromOptions(options);
- }
-
- @Test
- public void testInvalidNumberOfWorkerHarnessThreads() throws IOException {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setProject("foo-12345");
-
- options.setStagingLocation("gs://spam/ham/eggs");
- options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
- options.as(DataflowPipelineDebugOptions.class).setNumberOfWorkerHarnessThreads(-1);
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("Number of worker harness threads");
- thrown.expectMessage("Please make sure the value is non-negative.");
-
- DataflowPipelineRunner.fromOptions(options);
- }
-
- @Test
- public void testNoStagingLocationAndNoTempLocationFails() {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setProject("foo-project");
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage(
- "Missing required value: at least one of tempLocation or stagingLocation must be set.");
-
- DataflowPipelineRunner.fromOptions(options);
- }
-
- @Test
- public void testStagingLocationAndNoTempLocationSucceeds() throws Exception {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setGcpCredential(new TestCredential());
- options.setProject("foo-project");
- options.setStagingLocation("gs://spam/ham/eggs");
- options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
- DataflowPipelineRunner.fromOptions(options);
- }
-
- @Test
- public void testTempLocationAndNoStagingLocationSucceeds() throws Exception {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setGcpCredential(new TestCredential());
- options.setProject("foo-project");
- options.setTempLocation("gs://spam/ham/eggs");
- options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
-
- DataflowPipelineRunner.fromOptions(options);
- }
-
- @Test
- public void testInvalidJobName() throws IOException {
- List<String> invalidNames = Arrays.asList(
- "invalid_name",
- "0invalid",
- "invalid-");
- List<String> expectedReason = Arrays.asList(
- "JobName invalid",
- "JobName invalid",
- "JobName invalid");
-
- for (int i = 0; i < invalidNames.size(); ++i) {
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setJobName(invalidNames.get(i));
-
- try {
- DataflowPipelineRunner.fromOptions(options);
- fail("Expected IllegalArgumentException for jobName "
- + options.getJobName());
- } catch (IllegalArgumentException e) {
- assertThat(e.getMessage(),
- containsString(expectedReason.get(i)));
- }
- }
- }
-
- @Test
- public void testValidJobName() throws IOException {
- List<String> names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123",
- "this-one-is-fairly-long-01234567890123456789");
-
- for (String name : names) {
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setJobName(name);
-
- DataflowPipelineRunner runner = DataflowPipelineRunner
- .fromOptions(options);
- assertNotNull(runner);
- }
- }
-
- /**
- * A fake PTransform for testing.
- */
- public static class TestTransform
- extends PTransform<PCollection<Integer>, PCollection<Integer>> {
- public boolean translated = false;
-
- @Override
- public PCollection<Integer> apply(PCollection<Integer> input) {
- return PCollection.<Integer>createPrimitiveOutputInternal(
- input.getPipeline(),
- WindowingStrategy.globalDefault(),
- input.isBounded());
- }
-
- @Override
- protected Coder<?> getDefaultOutputCoder(PCollection<Integer> input) {
- return input.getCoder();
- }
- }
-
- @Test
- public void testTransformTranslatorMissing() throws IOException {
- // Test that we throw if we don't provide a translation.
- ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
-
- DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
- Pipeline p = Pipeline.create(options);
-
- p.apply(Create.of(Arrays.asList(1, 2, 3)))
- .apply(new TestTransform());
-
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage(Matchers.containsString("no translator registered"));
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
- assertValidJob(jobCaptor.getValue());
- }
-
- @Test
- public void testTransformTranslator() throws IOException {
- // Test that we can provide a custom translation
- DataflowPipelineOptions options = buildPipelineOptions();
- Pipeline p = Pipeline.create(options);
- TestTransform transform = new TestTransform();
-
- p.apply(Create.of(Arrays.asList(1, 2, 3)).withCoder(BigEndianIntegerCoder.of()))
- .apply(transform);
-
- DataflowPipelineTranslator translator = DataflowPipelineRunner
- .fromOptions(options).getTranslator();
-
- DataflowPipelineTranslator.registerTransformTranslator(
- TestTransform.class,
- new DataflowPipelineTranslator.TransformTranslator<TestTransform>() {
- @SuppressWarnings("unchecked")
- @Override
- public void translate(
- TestTransform transform,
- DataflowPipelineTranslator.TranslationContext context) {
- transform.translated = true;
-
- // Note: This is about the minimum needed to fake out a
- // translation. This obviously isn't a real translation.
- context.addStep(transform, "TestTranslate");
- context.addOutput("output", context.getOutput(transform));
- }
- });
-
- translator.translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
- assertTrue(transform.translated);
- }
-
- /** Records all the composite transforms visited within the Pipeline. */
- private static class CompositeTransformRecorder implements PipelineVisitor {
- private List<PTransform<?, ?>> transforms = new ArrayList<>();
-
- @Override
- public void enterCompositeTransform(TransformTreeNode node) {
- if (node.getTransform() != null) {
- transforms.add(node.getTransform());
- }
- }
-
- @Override
- public void leaveCompositeTransform(TransformTreeNode node) {
- }
-
- @Override
- public void visitTransform(TransformTreeNode node) {
- }
-
- @Override
- public void visitValue(PValue value, TransformTreeNode producer) {
- }
-
- public List<PTransform<?, ?>> getCompositeTransforms() {
- return transforms;
- }
- }
-
- @Test
- public void testApplyIsScopedToExactClass() throws IOException {
- DataflowPipelineOptions options = buildPipelineOptions();
- Pipeline p = Pipeline.create(options);
-
- Create.TimestampedValues<String> transform =
- Create.timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now())));
- p.apply(transform);
-
- CompositeTransformRecorder recorder = new CompositeTransformRecorder();
- p.traverseTopologically(recorder);
-
- // The recorder will also have seen a Create.Values composite as well, but we can't obtain that
- // transform.
- assertThat(
- "Expected to have seen CreateTimestamped composite transform.",
- recorder.getCompositeTransforms(),
- hasItem(transform));
- assertThat(
- "Expected to have two composites, CreateTimestamped and Create.Values",
- recorder.getCompositeTransforms(),
- hasItem(Matchers.<PTransform<?, ?>>isA((Class) Create.Values.class)));
- }
-
- @Test
- public void testToString() {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setJobName("TestJobName");
- options.setProject("test-project");
- options.setTempLocation("gs://test/temp/location");
- options.setGcpCredential(new TestCredential());
- options.setPathValidatorClass(NoopPathValidator.class);
- assertEquals(
- "DataflowPipelineRunner#testjobname",
- DataflowPipelineRunner.fromOptions(options).toString());
- }
-
- private static PipelineOptions makeOptions(boolean streaming) {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setStreaming(streaming);
- options.setJobName("TestJobName");
- options.setProject("test-project");
- options.setTempLocation("gs://test/temp/location");
- options.setGcpCredential(new TestCredential());
- options.setPathValidatorClass(NoopPathValidator.class);
- return options;
- }
-
- private void testUnsupportedSource(PTransform<PInput, ?> source, String name, boolean streaming)
- throws Exception {
- String mode = streaming ? "streaming" : "batch";
- thrown.expect(UnsupportedOperationException.class);
- thrown.expectMessage(
- "The DataflowPipelineRunner in " + mode + " mode does not support " + name);
-
- Pipeline p = Pipeline.create(makeOptions(streaming));
- p.apply(source);
- p.run();
- }
-
- @Test
- public void testBoundedSourceUnsupportedInStreaming() throws Exception {
- testUnsupportedSource(
- AvroSource.readFromFileWithClass("foo", String.class), "Read.Bounded", true);
- }
-
- @Test
- public void testBigQueryIOSourceUnsupportedInStreaming() throws Exception {
- testUnsupportedSource(
- BigQueryIO.Read.from("project:bar.baz").withoutValidation(), "BigQueryIO.Read", true);
- }
-
- @Test
- public void testAvroIOSourceUnsupportedInStreaming() throws Exception {
- testUnsupportedSource(
- AvroIO.Read.from("foo"), "AvroIO.Read", true);
- }
-
- @Test
- public void testTextIOSourceUnsupportedInStreaming() throws Exception {
- testUnsupportedSource(TextIO.Read.from("foo"), "TextIO.Read", true);
- }
-
- @Test
- public void testReadBoundedSourceUnsupportedInStreaming() throws Exception {
- testUnsupportedSource(Read.from(AvroSource.from("/tmp/test")), "Read.Bounded", true);
- }
-
- @Test
- public void testReadUnboundedUnsupportedInBatch() throws Exception {
- testUnsupportedSource(Read.from(new TestCountingSource(1)), "Read.Unbounded", false);
- }
-
- private void testUnsupportedSink(
- PTransform<PCollection<String>, PDone> sink, String name, boolean streaming)
- throws Exception {
- thrown.expect(UnsupportedOperationException.class);
- thrown.expectMessage(
- "The DataflowPipelineRunner in streaming mode does not support " + name);
-
- Pipeline p = Pipeline.create(makeOptions(streaming));
- p.apply(Create.of("foo")).apply(sink);
- p.run();
- }
-
- @Test
- public void testAvroIOSinkUnsupportedInStreaming() throws Exception {
- testUnsupportedSink(AvroIO.Write.to("foo").withSchema(String.class), "AvroIO.Write", true);
- }
-
- @Test
- public void testTextIOSinkUnsupportedInStreaming() throws Exception {
- testUnsupportedSink(TextIO.Write.to("foo"), "TextIO.Write", true);
- }
-
- @Test
- public void testBatchViewAsListToIsmRecordForGlobalWindow() throws Exception {
- DoFnTester<String, IsmRecord<WindowedValue<String>>> doFnTester =
- DoFnTester.of(new BatchViewAsList.ToIsmRecordForGlobalWindowDoFn<String>());
-
- // The order of the output elements is important relative to processing order
- assertThat(doFnTester.processBatch(ImmutableList.of("a", "b", "c")), contains(
- IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 0L), valueInGlobalWindow("a")),
- IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 1L), valueInGlobalWindow("b")),
- IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 2L), valueInGlobalWindow("c"))));
- }
-
- @Test
- public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception {
- DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>,
- IsmRecord<WindowedValue<Long>>> doFnTester =
- DoFnTester.of(
- new BatchViewAsList.ToIsmRecordForNonGlobalWindowDoFn<Long, IntervalWindow>(
- IntervalWindow.getCoder()));
-
- IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
- IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
- IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
- Iterable<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>> inputElements =
- ImmutableList.of(
- KV.of(1, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
- KV.of(
- windowA, WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
- KV.of(
- windowA, WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
- KV.of(
- windowA, WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
- KV.of(
- windowB, WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
- KV.of(
- windowB, WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING))
- )),
- KV.of(2, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
- KV.of(
- windowC, WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))
- )));
-
- // The order of the output elements is important relative to processing order
- assertThat(doFnTester.processBatch(inputElements), contains(
- IsmRecord.of(ImmutableList.of(windowA, 0L),
- WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
- IsmRecord.of(ImmutableList.of(windowA, 1L),
- WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
- IsmRecord.of(ImmutableList.of(windowA, 2L),
- WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
- IsmRecord.of(ImmutableList.of(windowB, 0L),
- WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
- IsmRecord.of(ImmutableList.of(windowB, 1L),
- WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)),
- IsmRecord.of(ImmutableList.of(windowC, 0L),
- WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))));
- }
-
- @Test
- public void testToIsmRecordForMapLikeDoFn() throws Exception {
- TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
- TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
- Coder<Long> keyCoder = VarLongCoder.of();
- Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
- IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
- 1,
- 2,
- ImmutableList.<Coder<?>>of(
- MetadataKeyCoder.of(keyCoder),
- IntervalWindow.getCoder(),
- BigEndianLongCoder.of()),
- FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
- DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
- IsmRecord<WindowedValue<Long>>> doFnTester =
- DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
- outputForSizeTag,
- outputForEntrySetTag,
- windowCoder,
- keyCoder,
- ismCoder,
- false /* unique keys */));
- doFnTester.setSideOutputTags(TupleTagList.of(
- ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
- IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
- IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
- IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
- Iterable<KV<Integer,
- Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
- ImmutableList.of(
- KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
- KV.of(KV.of(1L, windowA),
- WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
- // same window same key as to previous
- KV.of(KV.of(1L, windowA),
- WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
- // same window different key as to previous
- KV.of(KV.of(2L, windowA),
- WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
- // different window same key as to previous
- KV.of(KV.of(2L, windowB),
- WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
- // different window and different key as to previous
- KV.of(KV.of(3L, windowB),
- WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)))),
- KV.of(2, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
- // different shard
- KV.of(KV.of(4L, windowC),
- WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))));
-
- // The order of the output elements is important relative to processing order
- assertThat(doFnTester.processBatch(inputElements), contains(
- IsmRecord.of(
- ImmutableList.of(1L, windowA, 0L),
- WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
- IsmRecord.of(
- ImmutableList.of(1L, windowA, 1L),
- WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
- IsmRecord.of(
- ImmutableList.of(2L, windowA, 0L),
- WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
- IsmRecord.of(
- ImmutableList.of(2L, windowB, 0L),
- WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
- IsmRecord.of(
- ImmutableList.of(3L, windowB, 0L),
- WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
- IsmRecord.of(
- ImmutableList.of(4L, windowC, 0L),
- WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING))));
-
- // Verify the number of unique keys per window.
- assertThat(doFnTester.takeSideOutputElements(outputForSizeTag), contains(
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
- KV.of(windowA, 2L)),
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
- KV.of(windowB, 2L)),
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
- KV.of(windowC, 1L))
- ));
-
- // Verify the output for the unique keys.
- assertThat(doFnTester.takeSideOutputElements(outputForEntrySetTag), contains(
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
- KV.of(windowA, 1L)),
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
- KV.of(windowA, 2L)),
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
- KV.of(windowB, 2L)),
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
- KV.of(windowB, 3L)),
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
- KV.of(windowC, 4L))
- ));
- }
-
- @Test
- public void testToIsmRecordForMapLikeDoFnWithoutUniqueKeysThrowsException() throws Exception {
- TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
- TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
- Coder<Long> keyCoder = VarLongCoder.of();
- Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
- IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
- 1,
- 2,
- ImmutableList.<Coder<?>>of(
- MetadataKeyCoder.of(keyCoder),
- IntervalWindow.getCoder(),
- BigEndianLongCoder.of()),
- FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
- DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
- IsmRecord<WindowedValue<Long>>> doFnTester =
- DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
- outputForSizeTag,
- outputForEntrySetTag,
- windowCoder,
- keyCoder,
- ismCoder,
- true /* unique keys */));
- doFnTester.setSideOutputTags(TupleTagList.of(
- ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
- IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
-
- Iterable<KV<Integer,
- Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
- ImmutableList.of(
- KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
- KV.of(KV.of(1L, windowA),
- WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
- // same window same key as to previous
- KV.of(KV.of(1L, windowA),
- WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)))));
-
- try {
- doFnTester.processBatch(inputElements);
- fail("Expected UserCodeException");
- } catch (UserCodeException e) {
- assertTrue(e.getCause() instanceof IllegalStateException);
- IllegalStateException rootCause = (IllegalStateException) e.getCause();
- assertThat(rootCause.getMessage(), containsString("Unique keys are expected but found key"));
- }
- }
-
- @Test
- public void testToIsmMetadataRecordForSizeDoFn() throws Exception {
- TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
- TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
- Coder<Long> keyCoder = VarLongCoder.of();
- Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
- IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
- 1,
- 2,
- ImmutableList.<Coder<?>>of(
- MetadataKeyCoder.of(keyCoder),
- IntervalWindow.getCoder(),
- BigEndianLongCoder.of()),
- FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
- DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
- IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
- new BatchViewAsMultimap.ToIsmMetadataRecordForSizeDoFn<Long, Long, IntervalWindow>(
- windowCoder));
- doFnTester.setSideOutputTags(TupleTagList.of(
- ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
- IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
- IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
- IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
- Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
- ImmutableList.of(
- KV.of(1,
- (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
- KV.of(windowA, 2L),
- KV.of(windowA, 3L),
- KV.of(windowB, 7L))),
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
- (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
- KV.of(windowC, 9L))));
-
- // The order of the output elements is important relative to processing order
- assertThat(doFnTester.processBatch(inputElements), contains(
- IsmRecord.<WindowedValue<Long>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 0L),
- CoderUtils.encodeToByteArray(VarLongCoder.of(), 5L)),
- IsmRecord.<WindowedValue<Long>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 0L),
- CoderUtils.encodeToByteArray(VarLongCoder.of(), 7L)),
- IsmRecord.<WindowedValue<Long>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 0L),
- CoderUtils.encodeToByteArray(VarLongCoder.of(), 9L))
- ));
- }
-
- @Test
- public void testToIsmMetadataRecordForKeyDoFn() throws Exception {
- TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
- TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
-
- Coder<Long> keyCoder = VarLongCoder.of();
- Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
- IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
- 1,
- 2,
- ImmutableList.<Coder<?>>of(
- MetadataKeyCoder.of(keyCoder),
- IntervalWindow.getCoder(),
- BigEndianLongCoder.of()),
- FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
-
- DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
- IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
- new BatchViewAsMultimap.ToIsmMetadataRecordForKeyDoFn<Long, Long, IntervalWindow>(
- keyCoder, windowCoder));
- doFnTester.setSideOutputTags(TupleTagList.of(
- ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
-
- IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
- IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
- IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
- Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
- ImmutableList.of(
- KV.of(1,
- (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
- KV.of(windowA, 2L),
- // same window as previous
- KV.of(windowA, 3L),
- // different window as previous
- KV.of(windowB, 3L))),
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
- (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
- KV.of(windowC, 3L))));
-
- // The order of the output elements is important relative to processing order
- assertThat(doFnTester.processBatch(inputElements), contains(
- IsmRecord.<WindowedValue<Long>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 1L),
- CoderUtils.encodeToByteArray(VarLongCoder.of(), 2L)),
- IsmRecord.<WindowedValue<Long>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 2L),
- CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
- IsmRecord.<WindowedValue<Long>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 1L),
- CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
- IsmRecord.<WindowedValue<Long>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 1L),
- CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L))
- ));
- }
-
- @Test
- public void testToMapDoFn() throws Exception {
- Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
- DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
- IsmRecord<WindowedValue<TransformedMap<Long,
- WindowedValue<Long>,
- Long>>>> doFnTester =
- DoFnTester.of(new BatchViewAsMap.ToMapDoFn<Long, Long, IntervalWindow>(windowCoder));
-
-
- IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
- IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
- IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
- Iterable<KV<Integer,
- Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
- ImmutableList.of(
- KV.of(1,
- (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
- KV.of(windowA, WindowedValue.of(
- KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
- KV.of(windowA, WindowedValue.of(
- KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
- KV.of(windowB, WindowedValue.of(
- KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
- KV.of(windowB, WindowedValue.of(
- KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
- KV.of(2,
- (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
- KV.of(windowC, WindowedValue.of(
- KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
-
- // The order of the output elements is important relative to processing order
- List<IsmRecord<WindowedValue<TransformedMap<Long,
- WindowedValue<Long>,
- Long>>>> output =
- doFnTester.processBatch(inputElements);
- assertEquals(3, output.size());
- Map<Long, Long> outputMap;
-
- outputMap = output.get(0).getValue().getValue();
- assertEquals(2, outputMap.size());
- assertEquals(ImmutableMap.of(1L, 11L, 2L, 21L), outputMap);
-
- outputMap = output.get(1).getValue().getValue();
- assertEquals(2, outputMap.size());
- assertEquals(ImmutableMap.of(2L, 21L, 3L, 31L), outputMap);
-
- outputMap = output.get(2).getValue().getValue();
- assertEquals(1, outputMap.size());
- assertEquals(ImmutableMap.of(4L, 41L), outputMap);
- }
-
- @Test
- public void testToMultimapDoFn() throws Exception {
- Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
-
- DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
- IsmRecord<WindowedValue<TransformedMap<Long,
- Iterable<WindowedValue<Long>>,
- Iterable<Long>>>>> doFnTester =
- DoFnTester.of(
- new BatchViewAsMultimap.ToMultimapDoFn<Long, Long, IntervalWindow>(windowCoder));
-
-
- IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
- IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
- IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
-
- Iterable<KV<Integer,
- Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
- ImmutableList.of(
- KV.of(1,
- (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
- KV.of(windowA, WindowedValue.of(
- KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
- KV.of(windowA, WindowedValue.of(
- KV.of(1L, 12L), new Instant(5), windowA, PaneInfo.NO_FIRING)),
- KV.of(windowA, WindowedValue.of(
- KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
- KV.of(windowB, WindowedValue.of(
- KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
- KV.of(windowB, WindowedValue.of(
- KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
- KV.of(2,
- (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
- KV.of(windowC, WindowedValue.of(
- KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
-
- // The order of the output elements is important relative to processing order
- List<IsmRecord<WindowedValue<TransformedMap<Long,
- Iterable<WindowedValue<Long>>,
- Iterable<Long>>>>> output =
- doFnTester.processBatch(inputElements);
- assertEquals(3, output.size());
- Map<Long, Iterable<Long>> outputMap;
-
- outputMap = output.get(0).getValue().getValue();
- assertEquals(2, outputMap.size());
- assertThat(outputMap.get(1L), containsInAnyOrder(11L, 12L));
- assertThat(outputMap.get(2L), containsInAnyOrder(21L));
-
- outputMap = output.get(1).getValue().getValue();
- assertEquals(2, outputMap.size());
- assertThat(outputMap.get(2L), containsInAnyOrder(21L));
- assertThat(outputMap.get(3L), containsInAnyOrder(31L));
-
- outputMap = output.get(2).getValue().getValue();
- assertEquals(1, outputMap.size());
- assertThat(outputMap.get(4L), containsInAnyOrder(41L));
- }
-}
[14/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java
new file mode 100644
index 0000000..71c8a78
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java
@@ -0,0 +1,247 @@
+/*
+ * 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.dataflow.options;
+
+import org.apache.beam.runners.dataflow.util.DataflowPathValidator;
+import org.apache.beam.runners.dataflow.util.DataflowTransport;
+import org.apache.beam.runners.dataflow.util.GcsStager;
+import org.apache.beam.runners.dataflow.util.Stager;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Hidden;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.util.PathValidator;
+
+import com.google.api.services.dataflow.Dataflow;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Internal. Options used to control execution of the Dataflow SDK for
+ * debugging and testing purposes.
+ */
+@Description("[Internal] Options used to control execution of the Dataflow SDK for "
+ + "debugging and testing purposes.")
+@Hidden
+public interface DataflowPipelineDebugOptions extends PipelineOptions {
+
+ /**
+ * The list of backend experiments to enable.
+ *
+ * <p>Dataflow provides a number of experimental features that can be enabled
+ * with this flag.
+ *
+ * <p>Please sync with the Dataflow team before enabling any experiments.
+ */
+ @Description("[Experimental] Dataflow provides a number of experimental features that can "
+ + "be enabled with this flag. Please sync with the Dataflow team before enabling any "
+ + "experiments.")
+ @Experimental
+ List<String> getExperiments();
+ void setExperiments(List<String> value);
+
+ /**
+ * The root URL for the Dataflow API. {@code dataflowEndpoint} can override this value
+ * if it contains an absolute URL, otherwise {@code apiRootUrl} will be combined with
+ * {@code dataflowEndpoint} to generate the full URL to communicate with the Dataflow API.
+ */
+ @Description("The root URL for the Dataflow API. dataflowEndpoint can override this "
+ + "value if it contains an absolute URL, otherwise apiRootUrl will be combined with "
+ + "dataflowEndpoint to generate the full URL to communicate with the Dataflow API.")
+ @Default.String(Dataflow.DEFAULT_ROOT_URL)
+ String getApiRootUrl();
+ void setApiRootUrl(String value);
+
+ /**
+ * Dataflow endpoint to use.
+ *
+ * <p>Defaults to the current version of the Google Cloud Dataflow
+ * API, at the time the current SDK version was released.
+ *
+ * <p>If the string contains "://", then this is treated as a URL,
+ * otherwise {@link #getApiRootUrl()} is used as the root
+ * URL.
+ */
+ @Description("The URL for the Dataflow API. If the string contains \"://\", this"
+ + " will be treated as the entire URL, otherwise will be treated relative to apiRootUrl.")
+ @Default.String(Dataflow.DEFAULT_SERVICE_PATH)
+ String getDataflowEndpoint();
+ void setDataflowEndpoint(String value);
+
+ /**
+ * The path to write the translated Dataflow job specification out to
+ * at job submission time. The Dataflow job specification will be represented in JSON
+ * format.
+ */
+ @Description("The path to write the translated Dataflow job specification out to "
+ + "at job submission time. The Dataflow job specification will be represented in JSON "
+ + "format.")
+ String getDataflowJobFile();
+ void setDataflowJobFile(String value);
+
+ /**
+ * The class of the validator that should be created and used to validate paths.
+ * If pathValidator has not been set explicitly, an instance of this class will be
+ * constructed and used as the path validator.
+ */
+ @Description("The class of the validator that should be created and used to validate paths. "
+ + "If pathValidator has not been set explicitly, an instance of this class will be "
+ + "constructed and used as the path validator.")
+ @Default.Class(DataflowPathValidator.class)
+ Class<? extends PathValidator> getPathValidatorClass();
+ void setPathValidatorClass(Class<? extends PathValidator> validatorClass);
+
+ /**
+ * The path validator instance that should be used to validate paths.
+ * If no path validator has been set explicitly, the default is to use the instance factory that
+ * constructs a path validator based upon the currently set pathValidatorClass.
+ */
+ @JsonIgnore
+ @Description("The path validator instance that should be used to validate paths. "
+ + "If no path validator has been set explicitly, the default is to use the instance factory "
+ + "that constructs a path validator based upon the currently set pathValidatorClass.")
+ @Default.InstanceFactory(PathValidatorFactory.class)
+ PathValidator getPathValidator();
+ void setPathValidator(PathValidator validator);
+
+ /**
+ * The class responsible for staging resources to be accessible by workers
+ * during job execution. If stager has not been set explicitly, an instance of this class
+ * will be created and used as the resource stager.
+ */
+ @Description("The class of the stager that should be created and used to stage resources. "
+ + "If stager has not been set explicitly, an instance of the this class will be created "
+ + "and used as the resource stager.")
+ @Default.Class(GcsStager.class)
+ Class<? extends Stager> getStagerClass();
+ void setStagerClass(Class<? extends Stager> stagerClass);
+
+ /**
+ * The resource stager instance that should be used to stage resources.
+ * If no stager has been set explicitly, the default is to use the instance factory
+ * that constructs a resource stager based upon the currently set stagerClass.
+ */
+ @JsonIgnore
+ @Description("The resource stager instance that should be used to stage resources. "
+ + "If no stager has been set explicitly, the default is to use the instance factory "
+ + "that constructs a resource stager based upon the currently set stagerClass.")
+ @Default.InstanceFactory(StagerFactory.class)
+ Stager getStager();
+ void setStager(Stager stager);
+
+ /**
+ * An instance of the Dataflow client. Defaults to creating a Dataflow client
+ * using the current set of options.
+ */
+ @JsonIgnore
+ @Description("An instance of the Dataflow client. Defaults to creating a Dataflow client "
+ + "using the current set of options.")
+ @Default.InstanceFactory(DataflowClientFactory.class)
+ Dataflow getDataflowClient();
+ void setDataflowClient(Dataflow value);
+
+ /** Returns the default Dataflow client built from the passed in PipelineOptions. */
+ public static class DataflowClientFactory implements DefaultValueFactory<Dataflow> {
+ @Override
+ public Dataflow create(PipelineOptions options) {
+ return DataflowTransport.newDataflowClient(
+ options.as(DataflowPipelineOptions.class)).build();
+ }
+ }
+
+ /**
+ * Mapping of old PTranform names to new ones, specified as JSON
+ * <code>{"oldName":"newName",...}</code>. To mark a transform as deleted, make newName the
+ * empty string.
+ */
+ @JsonIgnore
+ @Description(
+ "Mapping of old PTranform names to new ones, specified as JSON "
+ + "{\"oldName\":\"newName\",...}. To mark a transform as deleted, make newName the empty "
+ + "string.")
+ Map<String, String> getTransformNameMapping();
+ void setTransformNameMapping(Map<String, String> value);
+
+ /**
+ * Custom windmill_main binary to use with the streaming runner.
+ */
+ @Description("Custom windmill_main binary to use with the streaming runner")
+ String getOverrideWindmillBinary();
+ void setOverrideWindmillBinary(String value);
+
+ /**
+ * Number of threads to use on the Dataflow worker harness. If left unspecified,
+ * the Dataflow service will compute an appropriate number of threads to use.
+ */
+ @Description("Number of threads to use on the Dataflow worker harness. If left unspecified, "
+ + "the Dataflow service will compute an appropriate number of threads to use.")
+ int getNumberOfWorkerHarnessThreads();
+ void setNumberOfWorkerHarnessThreads(int value);
+
+ /**
+ * If {@literal true}, save a heap dump before killing a thread or process which is GC
+ * thrashing or out of memory. The location of the heap file will either be echoed back
+ * to the user, or the user will be given the opportunity to download the heap file.
+ *
+ * <p>
+ * CAUTION: Heap dumps can of comparable size to the default boot disk. Consider increasing
+ * the boot disk size before setting this flag to true.
+ */
+ @Description("If {@literal true}, save a heap dump before killing a thread or process "
+ + "which is GC thrashing or out of memory.")
+ boolean getDumpHeapOnOOM();
+ void setDumpHeapOnOOM(boolean dumpHeapBeforeExit);
+
+ /**
+ * Creates a {@link PathValidator} object using the class specified in
+ * {@link #getPathValidatorClass()}.
+ */
+ public static class PathValidatorFactory implements DefaultValueFactory<PathValidator> {
+ @Override
+ public PathValidator create(PipelineOptions options) {
+ DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class);
+ return InstanceBuilder.ofType(PathValidator.class)
+ .fromClass(debugOptions.getPathValidatorClass())
+ .fromFactoryMethod("fromOptions")
+ .withArg(PipelineOptions.class, options)
+ .build();
+ }
+ }
+
+ /**
+ * Creates a {@link Stager} object using the class specified in
+ * {@link #getStagerClass()}.
+ */
+ public static class StagerFactory implements DefaultValueFactory<Stager> {
+ @Override
+ public Stager create(PipelineOptions options) {
+ DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class);
+ return InstanceBuilder.ofType(Stager.class)
+ .fromClass(debugOptions.getStagerClass())
+ .fromFactoryMethod("fromOptions")
+ .withArg(PipelineOptions.class, options)
+ .build();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
new file mode 100644
index 0000000..6e6ad96
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
@@ -0,0 +1,126 @@
+/*
+ * 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.dataflow.options;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.sdk.options.ApplicationNameOptions;
+import org.apache.beam.sdk.options.BigQueryOptions;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PubsubOptions;
+import org.apache.beam.sdk.options.StreamingOptions;
+import org.apache.beam.sdk.options.Validation;
+
+import com.google.common.base.MoreObjects;
+
+import org.joda.time.DateTimeUtils;
+import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+/**
+ * Options that can be used to configure the {@link DataflowPipelineRunner}.
+ */
+@Description("Options that configure the Dataflow pipeline.")
+public interface DataflowPipelineOptions
+ extends PipelineOptions, GcpOptions, ApplicationNameOptions, DataflowPipelineDebugOptions,
+ DataflowPipelineWorkerPoolOptions, BigQueryOptions, GcsOptions, StreamingOptions,
+ CloudDebuggerOptions, DataflowWorkerLoggingOptions, DataflowProfilingOptions,
+ PubsubOptions {
+
+ @Description("Project id. Required when running a Dataflow in the cloud. "
+ + "See https://cloud.google.com/storage/docs/projects for further details.")
+ @Override
+ @Validation.Required
+ @Default.InstanceFactory(DefaultProjectFactory.class)
+ String getProject();
+ @Override
+ void setProject(String value);
+
+ /**
+ * GCS path for staging local files, e.g. gs://bucket/object
+ *
+ * <p>Must be a valid Cloud Storage URL, beginning with the prefix "gs://"
+ *
+ * <p>At least one of {@link PipelineOptions#getTempLocation()} or {@link #getStagingLocation()}
+ * must be set. If {@link #getStagingLocation()} is not set, then the Dataflow
+ * pipeline defaults to using {@link PipelineOptions#getTempLocation()}.
+ */
+ @Description("GCS path for staging local files, e.g. \"gs://bucket/object\". "
+ + "Must be a valid Cloud Storage URL, beginning with the prefix \"gs://\". "
+ + "At least one of stagingLocation or tempLocation must be set. If stagingLocation is unset, "
+ + "defaults to using tempLocation.")
+ String getStagingLocation();
+ void setStagingLocation(String value);
+
+ /**
+ * The Dataflow job name is used as an idempotence key within the Dataflow service.
+ * If there is an existing job that is currently active, another active job with the same
+ * name will not be able to be created. Defaults to using the ApplicationName-UserName-Date.
+ */
+ @Description("The Dataflow job name is used as an idempotence key within the Dataflow service. "
+ + "For each running job in the same GCP project, jobs with the same name cannot be created "
+ + "unless the new job is an explicit update of the previous one. Defaults to using "
+ + "ApplicationName-UserName-Date. The job name must match the regular expression "
+ + "'[a-z]([-a-z0-9]{0,38}[a-z0-9])?'. The runner will automatically truncate the name of the "
+ + "job and convert to lower case.")
+ @Default.InstanceFactory(JobNameFactory.class)
+ String getJobName();
+ void setJobName(String value);
+
+ /**
+ * Whether to update the currently running pipeline with the same name as this one.
+ */
+ @Description(
+ "If set, replace the existing pipeline with the name specified by --jobName with "
+ + "this pipeline, preserving state.")
+ boolean isUpdate();
+ void setUpdate(boolean value);
+
+ /**
+ * Returns a normalized job name constructed from {@link ApplicationNameOptions#getAppName()}, the
+ * local system user name (if available), and the current time. The normalization makes sure that
+ * the job name matches the required pattern of [a-z]([-a-z0-9]*[a-z0-9])? and length limit of 40
+ * characters.
+ *
+ * <p>This job name factory is only able to generate one unique name per second per application
+ * and user combination.
+ */
+ public static class JobNameFactory implements DefaultValueFactory<String> {
+ private static final DateTimeFormatter FORMATTER =
+ DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC);
+
+ @Override
+ public String create(PipelineOptions options) {
+ String appName = options.as(ApplicationNameOptions.class).getAppName();
+ String normalizedAppName = appName == null || appName.length() == 0 ? "dataflow"
+ : appName.toLowerCase()
+ .replaceAll("[^a-z0-9]", "0")
+ .replaceAll("^[^a-z]", "a");
+ String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), "");
+ String normalizedUserName = userName.toLowerCase()
+ .replaceAll("[^a-z0-9]", "0");
+ String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis());
+ return normalizedAppName + "-" + normalizedUserName + "-" + datePart;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java
new file mode 100644
index 0000000..a29b328
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineWorkerPoolOptions.java
@@ -0,0 +1,263 @@
+/*
+ * 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.dataflow.options;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Hidden;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.util.List;
+
+/**
+ * Options that are used to configure the Dataflow pipeline worker pool.
+ */
+@Description("Options that are used to configure the Dataflow pipeline worker pool.")
+public interface DataflowPipelineWorkerPoolOptions extends PipelineOptions {
+ /**
+ * Number of workers to use when executing the Dataflow job. Note that selection of an autoscaling
+ * algorithm other then {@code NONE} will affect the size of the worker pool. If left unspecified,
+ * the Dataflow service will determine the number of workers.
+ */
+ @Description("Number of workers to use when executing the Dataflow job. Note that "
+ + "selection of an autoscaling algorithm other then \"NONE\" will affect the "
+ + "size of the worker pool. If left unspecified, the Dataflow service will "
+ + "determine the number of workers.")
+ int getNumWorkers();
+ void setNumWorkers(int value);
+
+ /**
+ * Type of autoscaling algorithm to use.
+ */
+ @Experimental(Experimental.Kind.AUTOSCALING)
+ public enum AutoscalingAlgorithmType {
+ /** Use numWorkers machines. Do not autoscale the worker pool. */
+ NONE("AUTOSCALING_ALGORITHM_NONE"),
+
+ @Deprecated
+ BASIC("AUTOSCALING_ALGORITHM_BASIC"),
+
+ /** Autoscale the workerpool based on throughput (up to maxNumWorkers). */
+ THROUGHPUT_BASED("AUTOSCALING_ALGORITHM_BASIC");
+
+ private final String algorithm;
+
+ private AutoscalingAlgorithmType(String algorithm) {
+ this.algorithm = algorithm;
+ }
+
+ /** Returns the string representation of this type. */
+ public String getAlgorithm() {
+ return this.algorithm;
+ }
+ }
+
+ /**
+ * [Experimental] The autoscaling algorithm to use for the workerpool.
+ *
+ * <ul>
+ * <li>NONE: does not change the size of the worker pool.</li>
+ * <li>BASIC: autoscale the worker pool size up to maxNumWorkers until the job completes.</li>
+ * <li>THROUGHPUT_BASED: autoscale the workerpool based on throughput (up to maxNumWorkers).
+ * </li>
+ * </ul>
+ */
+ @Description("[Experimental] The autoscaling algorithm to use for the workerpool. "
+ + "NONE: does not change the size of the worker pool. "
+ + "BASIC (deprecated): autoscale the worker pool size up to maxNumWorkers until the job "
+ + "completes. "
+ + "THROUGHPUT_BASED: autoscale the workerpool based on throughput (up to maxNumWorkers).")
+ @Experimental(Experimental.Kind.AUTOSCALING)
+ AutoscalingAlgorithmType getAutoscalingAlgorithm();
+ void setAutoscalingAlgorithm(AutoscalingAlgorithmType value);
+
+ /**
+ * The maximum number of workers to use for the workerpool. This options limits the size of the
+ * workerpool for the lifetime of the job, including
+ * <a href="https://cloud.google.com/dataflow/pipelines/updating-a-pipeline">pipeline updates</a>.
+ * If left unspecified, the Dataflow service will compute a ceiling.
+ */
+ @Description("The maximum number of workers to use for the workerpool. This options limits the "
+ + "size of the workerpool for the lifetime of the job, including pipeline updates. "
+ + "If left unspecified, the Dataflow service will compute a ceiling.")
+ int getMaxNumWorkers();
+ void setMaxNumWorkers(int value);
+
+ /**
+ * Remote worker disk size, in gigabytes, or 0 to use the default size.
+ */
+ @Description("Remote worker disk size, in gigabytes, or 0 to use the default size.")
+ int getDiskSizeGb();
+ void setDiskSizeGb(int value);
+
+ /**
+ * Docker container image that executes Dataflow worker harness, residing in Google Container
+ * Registry.
+ */
+ @Default.InstanceFactory(WorkerHarnessContainerImageFactory.class)
+ @Description("Docker container image that executes Dataflow worker harness, residing in Google "
+ + " Container Registry.")
+ @Hidden
+ String getWorkerHarnessContainerImage();
+ void setWorkerHarnessContainerImage(String value);
+
+ /**
+ * Returns the default Docker container image that executes Dataflow worker harness, residing in
+ * Google Container Registry.
+ */
+ public static class WorkerHarnessContainerImageFactory
+ implements DefaultValueFactory<String> {
+ @Override
+ public String create(PipelineOptions options) {
+ DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
+ if (dataflowOptions.isStreaming()) {
+ return DataflowPipelineRunner.STREAMING_WORKER_HARNESS_CONTAINER_IMAGE;
+ } else {
+ return DataflowPipelineRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE;
+ }
+ }
+ }
+
+ /**
+ * GCE <a href="https://cloud.google.com/compute/docs/networking">network</a> for launching
+ * workers.
+ *
+ * <p>Default is up to the Dataflow service.
+ */
+ @Description("GCE network for launching workers. For more information, see the reference "
+ + "documentation https://cloud.google.com/compute/docs/networking. "
+ + "Default is up to the Dataflow service.")
+ String getNetwork();
+ void setNetwork(String value);
+
+ /**
+ * GCE <a href="https://cloud.google.com/compute/docs/networking">subnetwork</a> for launching
+ * workers.
+ *
+ * <p>Default is up to the Dataflow service. Expected format is
+ * regions/REGION/subnetworks/SUBNETWORK.
+ *
+ * <p>You may also need to specify network option.
+ */
+ @Description("GCE subnetwork for launching workers. For more information, see the reference "
+ + "documentation https://cloud.google.com/compute/docs/networking. "
+ + "Default is up to the Dataflow service.")
+ String getSubnetwork();
+ void setSubnetwork(String value);
+
+ /**
+ * GCE <a href="https://developers.google.com/compute/docs/zones"
+ * >availability zone</a> for launching workers.
+ *
+ * <p>Default is up to the Dataflow service.
+ */
+ @Description("GCE availability zone for launching workers. See "
+ + "https://developers.google.com/compute/docs/zones for a list of valid options. "
+ + "Default is up to the Dataflow service.")
+ String getZone();
+ void setZone(String value);
+
+ /**
+ * Machine type to create Dataflow worker VMs as.
+ *
+ * <p>See <a href="https://cloud.google.com/compute/docs/machine-types">GCE machine types</a>
+ * for a list of valid options.
+ *
+ * <p>If unset, the Dataflow service will choose a reasonable default.
+ */
+ @Description("Machine type to create Dataflow worker VMs as. See "
+ + "https://cloud.google.com/compute/docs/machine-types for a list of valid options. "
+ + "If unset, the Dataflow service will choose a reasonable default.")
+ String getWorkerMachineType();
+ void setWorkerMachineType(String value);
+
+ /**
+ * The policy for tearing down the workers spun up by the service.
+ */
+ public enum TeardownPolicy {
+ /**
+ * All VMs created for a Dataflow job are deleted when the job finishes, regardless of whether
+ * it fails or succeeds.
+ */
+ TEARDOWN_ALWAYS("TEARDOWN_ALWAYS"),
+ /**
+ * All VMs created for a Dataflow job are left running when the job finishes, regardless of
+ * whether it fails or succeeds.
+ */
+ TEARDOWN_NEVER("TEARDOWN_NEVER"),
+ /**
+ * All VMs created for a Dataflow job are deleted when the job succeeds, but are left running
+ * when it fails. (This is typically used for debugging failing jobs by SSHing into the
+ * workers.)
+ */
+ TEARDOWN_ON_SUCCESS("TEARDOWN_ON_SUCCESS");
+
+ private final String teardownPolicy;
+
+ private TeardownPolicy(String teardownPolicy) {
+ this.teardownPolicy = teardownPolicy;
+ }
+
+ public String getTeardownPolicyName() {
+ return this.teardownPolicy;
+ }
+ }
+
+ /**
+ * The teardown policy for the VMs.
+ *
+ * <p>If unset, the Dataflow service will choose a reasonable default.
+ */
+ @Description("The teardown policy for the VMs. If unset, the Dataflow service will "
+ + "choose a reasonable default.")
+ TeardownPolicy getTeardownPolicy();
+ void setTeardownPolicy(TeardownPolicy value);
+
+ /**
+ * List of local files to make available to workers.
+ *
+ * <p>Files are placed on the worker's classpath.
+ *
+ * <p>The default value is the list of jars from the main program's classpath.
+ */
+ @Description("Files to stage on GCS and make available to workers. "
+ + "Files are placed on the worker's classpath. "
+ + "The default value is all files from the classpath.")
+ @JsonIgnore
+ List<String> getFilesToStage();
+ void setFilesToStage(List<String> value);
+
+ /**
+ * Specifies what type of persistent disk should be used. The value should be a full or partial
+ * URL of a disk type resource, e.g., zones/us-central1-f/disks/pd-standard. For
+ * more information, see the
+ * <a href="https://cloud.google.com/compute/docs/reference/latest/diskTypes">API reference
+ * documentation for DiskTypes</a>.
+ */
+ @Description("Specifies what type of persistent disk should be used. The value should be a full "
+ + "or partial URL of a disk type resource, e.g., zones/us-central1-f/disks/pd-standard. For "
+ + "more information, see the API reference documentation for DiskTypes: "
+ + "https://cloud.google.com/compute/docs/reference/latest/diskTypes")
+ String getWorkerDiskType();
+ void setWorkerDiskType(String value);
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java
new file mode 100644
index 0000000..c7b4c91
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptions.java
@@ -0,0 +1,50 @@
+/*
+ * 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.dataflow.options;
+
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Hidden;
+
+import java.util.HashMap;
+
+/**
+ * Options for controlling profiling of pipeline execution.
+ */
+@Description("[Experimental] Used to configure profiling of the Dataflow pipeline")
+@Experimental
+@Hidden
+public interface DataflowProfilingOptions {
+
+ @Description("Whether to periodically dump profiling information to local disk.\n"
+ + "WARNING: Enabling this option may fill local disk with profiling information.")
+ boolean getEnableProfilingAgent();
+ void setEnableProfilingAgent(boolean enabled);
+
+ @Description(
+ "[INTERNAL] Additional configuration for the profiling agent. Not typically necessary.")
+ @Hidden
+ DataflowProfilingAgentConfiguration getProfilingAgentConfiguration();
+ void setProfilingAgentConfiguration(DataflowProfilingAgentConfiguration configuration);
+
+ /**
+ * Configuration the for profiling agent.
+ */
+ public static class DataflowProfilingAgentConfiguration extends HashMap<String, Object> {
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerHarnessOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerHarnessOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerHarnessOptions.java
new file mode 100644
index 0000000..e946e6b
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerHarnessOptions.java
@@ -0,0 +1,55 @@
+/*
+ * 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.dataflow.options;
+
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Hidden;
+
+/**
+ * Options that are used exclusively within the Dataflow worker harness.
+ * These options have no effect at pipeline creation time.
+ */
+@Description("[Internal] Options that are used exclusively within the Dataflow worker harness. "
+ + "These options have no effect at pipeline creation time.")
+@Hidden
+public interface DataflowWorkerHarnessOptions extends DataflowPipelineOptions {
+ /**
+ * The identity of the worker running this pipeline.
+ */
+ @Description("The identity of the worker running this pipeline.")
+ String getWorkerId();
+ void setWorkerId(String value);
+
+ /**
+ * The identity of the Dataflow job.
+ */
+ @Description("The identity of the Dataflow job.")
+ String getJobId();
+ void setJobId(String value);
+
+ /**
+ * The size of the worker's in-memory cache, in megabytes.
+ *
+ * <p>Currently, this cache is used for storing read values of side inputs.
+ */
+ @Description("The size of the worker's in-memory cache, in megabytes.")
+ @Default.Integer(100)
+ Integer getWorkerCacheMb();
+ void setWorkerCacheMb(Integer value);
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java
new file mode 100644
index 0000000..74ab59a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java
@@ -0,0 +1,159 @@
+/*
+ * 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.dataflow.options;
+
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import com.google.common.base.Preconditions;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Options that are used to control logging configuration on the Dataflow worker.
+ */
+@Description("Options that are used to control logging configuration on the Dataflow worker.")
+public interface DataflowWorkerLoggingOptions extends PipelineOptions {
+ /**
+ * The set of log levels that can be used on the Dataflow worker.
+ */
+ public enum Level {
+ DEBUG, ERROR, INFO, TRACE, WARN
+ }
+
+ /**
+ * This option controls the default log level of all loggers without a log level override.
+ */
+ @Description("Controls the default log level of all loggers without a log level override.")
+ @Default.Enum("INFO")
+ Level getDefaultWorkerLogLevel();
+ void setDefaultWorkerLogLevel(Level level);
+
+ /**
+ * This option controls the log levels for specifically named loggers.
+ *
+ * <p>Later options with equivalent names override earlier options.
+ *
+ * <p>See {@link WorkerLogLevelOverrides} for more information on how to configure logging
+ * on a per {@link Class}, {@link Package}, or name basis. If used from the command line,
+ * the expected format is {"Name":"Level",...}, further details on
+ * {@link WorkerLogLevelOverrides#from}.
+ */
+ @Description("This option controls the log levels for specifically named loggers. "
+ + "The expected format is {\"Name\":\"Level\",...}. The Dataflow worker uses "
+ + "java.util.logging, which supports a logging hierarchy based off of names that are '.' "
+ + "separated. For example, by specifying the value {\"a.b.c.Foo\":\"DEBUG\"}, the logger "
+ + "for the class 'a.b.c.Foo' will be configured to output logs at the DEBUG level. "
+ + "Similarly, by specifying the value {\"a.b.c\":\"WARN\"}, all loggers underneath the "
+ + "'a.b.c' package will be configured to output logs at the WARN level. Also, note that "
+ + "when multiple overrides are specified, the exact name followed by the closest parent "
+ + "takes precedence.")
+ WorkerLogLevelOverrides getWorkerLogLevelOverrides();
+ void setWorkerLogLevelOverrides(WorkerLogLevelOverrides value);
+
+ /**
+ * Defines a log level override for a specific class, package, or name.
+ *
+ * <p>{@code java.util.logging} is used on the Dataflow worker harness and supports
+ * a logging hierarchy based off of names that are "." separated. It is a common
+ * pattern to have the logger for a given class share the same name as the class itself.
+ * Given the classes {@code a.b.c.Foo}, {@code a.b.c.Xyz}, and {@code a.b.Bar}, with
+ * loggers named {@code "a.b.c.Foo"}, {@code "a.b.c.Xyz"}, and {@code "a.b.Bar"} respectively,
+ * we can override the log levels:
+ * <ul>
+ * <li>for {@code Foo} by specifying the name {@code "a.b.c.Foo"} or the {@link Class}
+ * representing {@code a.b.c.Foo}.
+ * <li>for {@code Foo}, {@code Xyz}, and {@code Bar} by specifying the name {@code "a.b"} or
+ * the {@link Package} representing {@code a.b}.
+ * <li>for {@code Foo} and {@code Bar} by specifying both of their names or classes.
+ * </ul>
+ * Note that by specifying multiple overrides, the exact name followed by the closest parent
+ * takes precedence.
+ */
+ public static class WorkerLogLevelOverrides extends HashMap<String, Level> {
+ /**
+ * Overrides the default log level for the passed in class.
+ *
+ * <p>This is equivalent to calling
+ * {@link #addOverrideForName(String, DataflowWorkerLoggingOptions.Level)}
+ * and passing in the {@link Class#getName() class name}.
+ */
+ public WorkerLogLevelOverrides addOverrideForClass(Class<?> klass, Level level) {
+ Preconditions.checkNotNull(klass, "Expected class to be not null.");
+ addOverrideForName(klass.getName(), level);
+ return this;
+ }
+
+ /**
+ * Overrides the default log level for the passed in package.
+ *
+ * <p>This is equivalent to calling
+ * {@link #addOverrideForName(String, DataflowWorkerLoggingOptions.Level)}
+ * and passing in the {@link Package#getName() package name}.
+ */
+ public WorkerLogLevelOverrides addOverrideForPackage(Package pkg, Level level) {
+ Preconditions.checkNotNull(pkg, "Expected package to be not null.");
+ addOverrideForName(pkg.getName(), level);
+ return this;
+ }
+
+ /**
+ * Overrides the default log level for the passed in name.
+ *
+ * <p>Note that because of the hierarchical nature of logger names, this will
+ * override the log level of all loggers that have the passed in name or
+ * a parent logger that has the passed in name.
+ */
+ public WorkerLogLevelOverrides addOverrideForName(String name, Level level) {
+ Preconditions.checkNotNull(name, "Expected name to be not null.");
+ Preconditions.checkNotNull(level,
+ "Expected level to be one of %s.", Arrays.toString(Level.values()));
+ put(name, level);
+ return this;
+ }
+
+ /**
+ * Expects a map keyed by logger {@code Name}s with values representing {@code Level}s.
+ * The {@code Name} generally represents the fully qualified Java
+ * {@link Class#getName() class name}, or fully qualified Java
+ * {@link Package#getName() package name}, or custom logger name. The {@code Level}
+ * represents the log level and must be one of {@link Level}.
+ */
+ @JsonCreator
+ public static WorkerLogLevelOverrides from(Map<String, String> values) {
+ Preconditions.checkNotNull(values, "Expected values to be not null.");
+ WorkerLogLevelOverrides overrides = new WorkerLogLevelOverrides();
+ for (Map.Entry<String, String> entry : values.entrySet()) {
+ try {
+ overrides.addOverrideForName(entry.getKey(), Level.valueOf(entry.getValue()));
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgumentException(String.format(
+ "Unsupported log level '%s' requested for %s. Must be one of %s.",
+ entry.getValue(), entry.getKey(), Arrays.toString(Level.values())));
+ }
+
+ }
+ return overrides;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineOptions.java
new file mode 100644
index 0000000..e66ffc9
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineOptions.java
@@ -0,0 +1,30 @@
+/*
+ * 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.dataflow.testing;
+
+import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+
+/**
+ * A set of options used to configure the {@link TestPipeline}.
+ */
+public interface TestDataflowPipelineOptions extends TestPipelineOptions,
+ BlockingDataflowPipelineOptions {
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..3ab91f5
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunner.java
@@ -0,0 +1,273 @@
+/*
+ * 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.dataflow.testing;
+
+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.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.JobMetrics;
+import com.google.api.services.dataflow.model.MetricUpdate;
+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;
+
+/**
+ * {@link TestDataflowPipelineRunner} is a pipeline runner that wraps a
+ * {@link DataflowPipelineRunner} when running tests against the {@link TestPipeline}.
+ *
+ * @see TestPipeline
+ */
+public class TestDataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob> {
+ private static final String TENTATIVE_COUNTER = "tentative";
+ private static final Logger LOG = LoggerFactory.getLogger(TestDataflowPipelineRunner.class);
+ private static final Map<String, PipelineResult> EXECUTION_RESULTS =
+ new ConcurrentHashMap<String, PipelineResult>();
+
+ private final TestDataflowPipelineOptions options;
+ private final DataflowPipelineRunner runner;
+ private int expectedNumberOfAssertions = 0;
+
+ TestDataflowPipelineRunner(TestDataflowPipelineOptions options) {
+ this.options = options;
+ this.runner = DataflowPipelineRunner.fromOptions(options);
+ }
+
+ /**
+ * Constructs a runner from the provided options.
+ */
+ public static TestDataflowPipelineRunner fromOptions(
+ PipelineOptions options) {
+ TestDataflowPipelineOptions dataflowOptions = options.as(TestDataflowPipelineOptions.class);
+ dataflowOptions.setStagingLocation(Joiner.on("/").join(
+ new String[]{dataflowOptions.getTempRoot(),
+ dataflowOptions.getJobName(), "output", "results"}));
+
+ 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);
+ }
+
+ DataflowPipelineJob run(Pipeline pipeline, DataflowPipelineRunner runner) {
+
+ final DataflowPipelineJob job;
+ try {
+ job = runner.run(pipeline);
+ } catch (DataflowJobExecutionException ex) {
+ throw new IllegalStateException("The dataflow failed.");
+ }
+
+ LOG.info("Running Dataflow job {} with {} expected assertions.",
+ job.getJobId(), expectedNumberOfAssertions);
+
+ CancelWorkflowOnError messageHandler = new CancelWorkflowOnError(
+ job, new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
+
+ try {
+ final Optional<Boolean> result;
+
+ if (options.isStreaming()) {
+ Future<Optional<Boolean>> resultFuture = options.getExecutorService().submit(
+ new Callable<Optional<Boolean>>() {
+ @Override
+ public Optional<Boolean> call() throws Exception {
+ try {
+ for (;;) {
+ Optional<Boolean> result = checkForSuccess(job);
+ if (result.isPresent()) {
+ return result;
+ }
+ Thread.sleep(10000L);
+ }
+ } finally {
+ LOG.info("Cancelling Dataflow job {}", job.getJobId());
+ job.cancel();
+ }
+ }
+ });
+ State finalState = job.waitToFinish(10L, TimeUnit.MINUTES, messageHandler);
+ if (finalState == null || finalState == State.RUNNING) {
+ LOG.info("Dataflow job {} took longer than 10 minutes to complete, cancelling.",
+ job.getJobId());
+ job.cancel();
+ }
+ result = resultFuture.get();
+ } else {
+ job.waitToFinish(-1, TimeUnit.SECONDS, messageHandler);
+ result = checkForSuccess(job);
+ }
+ if (!result.isPresent()) {
+ throw new IllegalStateException(
+ "The dataflow did not output a success or failure metric.");
+ } else if (!result.get()) {
+ throw new AssertionError(messageHandler.getErrorMessage() == null ?
+ "The dataflow did not return a failure reason."
+ : messageHandler.getErrorMessage());
+ }
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(e);
+ } catch (ExecutionException e) {
+ Throwables.propagateIfPossible(e.getCause());
+ throw new RuntimeException(e.getCause());
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ EXECUTION_RESULTS.put(options.getJobName(), job);
+ return job;
+ }
+
+ @Override
+ public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+ PTransform<InputT, OutputT> transform, InputT input) {
+ if (transform instanceof PAssert.OneSideInputAssert
+ || transform instanceof PAssert.TwoSideInputAssert) {
+ expectedNumberOfAssertions += 1;
+ }
+
+ return runner.apply(transform, input);
+ }
+
+ Optional<Boolean> checkForSuccess(DataflowPipelineJob job)
+ throws IOException {
+ State state = job.getState();
+ if (state == State.FAILED || state == State.CANCELLED) {
+ LOG.info("The pipeline failed");
+ return Optional.of(false);
+ }
+
+ JobMetrics metrics = job.getDataflowClient().projects().jobs()
+ .getMetrics(job.getProjectId(), job.getJobId()).execute();
+
+ if (metrics == null || metrics.getMetrics() == null) {
+ LOG.warn("Metrics not present for Dataflow job {}.", job.getJobId());
+ } else {
+ int successes = 0;
+ int failures = 0;
+ for (MetricUpdate metric : metrics.getMetrics()) {
+ if (metric.getName() == null || metric.getName().getContext() == null
+ || !metric.getName().getContext().containsKey(TENTATIVE_COUNTER)) {
+ // Don't double count using the non-tentative version of the metric.
+ continue;
+ }
+ if (PAssert.SUCCESS_COUNTER.equals(metric.getName().getName())) {
+ successes += ((BigDecimal) metric.getScalar()).intValue();
+ } else if (PAssert.FAILURE_COUNTER.equals(metric.getName().getName())) {
+ failures += ((BigDecimal) metric.getScalar()).intValue();
+ }
+ }
+
+ if (failures > 0) {
+ LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
+ + "{} expected assertions.", job.getJobId(), successes, failures,
+ expectedNumberOfAssertions);
+ return Optional.of(false);
+ } else if (successes >= expectedNumberOfAssertions) {
+ LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
+ + "{} expected assertions.", job.getJobId(), successes, failures,
+ expectedNumberOfAssertions);
+ return Optional.of(true);
+ }
+
+ LOG.info("Running Dataflow job {}. Found {} success, {} failures out of {} expected "
+ + "assertions.", job.getJobId(), successes, failures, expectedNumberOfAssertions);
+ }
+
+ return Optional.<Boolean>absent();
+ }
+
+ @Override
+ public String toString() {
+ return "TestDataflowPipelineRunner#" + options.getAppName();
+ }
+
+ /**
+ * Cancels the workflow on the first error message it sees.
+ *
+ * <p>Creates an error message representing the concatenation of all error messages seen.
+ */
+ private static class CancelWorkflowOnError implements JobMessagesHandler {
+ private final DataflowPipelineJob job;
+ private final JobMessagesHandler messageHandler;
+ private final StringBuffer errorMessage;
+ private CancelWorkflowOnError(DataflowPipelineJob job, JobMessagesHandler messageHandler) {
+ this.job = job;
+ this.messageHandler = messageHandler;
+ this.errorMessage = new StringBuffer();
+ }
+
+ @Override
+ public void process(List<JobMessage> messages) {
+ messageHandler.process(messages);
+ for (JobMessage message : messages) {
+ if (message.getMessageImportance() != null
+ && message.getMessageImportance().equals("JOB_MESSAGE_ERROR")) {
+ LOG.info("Dataflow job {} threw exception. Failure message was: {}",
+ job.getJobId(), message.getMessageText());
+ errorMessage.append(message.getMessageText());
+ }
+ }
+ if (errorMessage.length() > 0) {
+ LOG.info("Cancelling Dataflow job {}", job.getJobId());
+ try {
+ job.cancel();
+ } catch (Exception ignore) {
+ // The TestDataflowPipelineRunner will thrown an AssertionError with the job failure
+ // messages.
+ }
+ }
+ }
+
+ private String getErrorMessage() {
+ return errorMessage.toString();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowPathValidator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowPathValidator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowPathValidator.java
new file mode 100644
index 0000000..ddc5d6f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowPathValidator.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.dataflow.util;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.PathValidator;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+
+/**
+ * GCP implementation of {@link PathValidator}. Only GCS paths are allowed.
+ */
+public class DataflowPathValidator implements PathValidator {
+
+ private DataflowPipelineOptions dataflowOptions;
+
+ DataflowPathValidator(DataflowPipelineOptions options) {
+ this.dataflowOptions = options;
+ }
+
+ public static DataflowPathValidator fromOptions(PipelineOptions options) {
+ return new DataflowPathValidator(options.as(DataflowPipelineOptions.class));
+ }
+
+ /**
+ * Validates the the input GCS path is accessible and that the path
+ * is well formed.
+ */
+ @Override
+ public String validateInputFilePatternSupported(String filepattern) {
+ GcsPath gcsPath = getGcsPath(filepattern);
+ Preconditions.checkArgument(
+ dataflowOptions.getGcsUtil().isGcsPatternSupported(gcsPath.getObject()));
+ String returnValue = verifyPath(filepattern);
+ verifyPathIsAccessible(filepattern, "Could not find file %s");
+ return returnValue;
+ }
+
+ /**
+ * Validates the the output GCS path is accessible and that the path
+ * is well formed.
+ */
+ @Override
+ public String validateOutputFilePrefixSupported(String filePrefix) {
+ String returnValue = verifyPath(filePrefix);
+ verifyPathIsAccessible(filePrefix, "Output path does not exist or is not writeable: %s");
+ return returnValue;
+ }
+
+ @Override
+ public String verifyPath(String path) {
+ GcsPath gcsPath = getGcsPath(path);
+ Preconditions.checkArgument(gcsPath.isAbsolute(),
+ "Must provide absolute paths for Dataflow");
+ Preconditions.checkArgument(!gcsPath.getObject().contains("//"),
+ "Dataflow Service does not allow objects with consecutive slashes");
+ return gcsPath.toResourceName();
+ }
+
+ private void verifyPathIsAccessible(String path, String errorMessage) {
+ GcsPath gcsPath = getGcsPath(path);
+ try {
+ Preconditions.checkArgument(dataflowOptions.getGcsUtil().bucketExists(gcsPath),
+ errorMessage, path);
+ } catch (IOException e) {
+ throw new RuntimeException(
+ String.format("Unable to verify that GCS bucket gs://%s exists.", gcsPath.getBucket()),
+ e);
+ }
+ }
+
+ private GcsPath getGcsPath(String path) {
+ try {
+ return GcsPath.fromUri(path);
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgumentException(String.format(
+ "%s expected a valid 'gs://' path but was given '%s'",
+ dataflowOptions.getRunner().getSimpleName(), path), e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java
new file mode 100644
index 0000000..0199657
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java
@@ -0,0 +1,114 @@
+/*
+ * 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.dataflow.util;
+
+import static org.apache.beam.sdk.util.Transport.getJsonFactory;
+import static org.apache.beam.sdk.util.Transport.getTransport;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
+
+import com.google.api.client.auth.oauth2.Credential;
+import com.google.api.client.http.HttpRequestInitializer;
+import com.google.api.services.clouddebugger.v2.Clouddebugger;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
+import com.google.common.collect.ImmutableList;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+
+/**
+ * Helpers for cloud communication.
+ */
+public class DataflowTransport {
+
+
+ private static class ApiComponents {
+ public String rootUrl;
+ public String servicePath;
+
+ public ApiComponents(String root, String path) {
+ this.rootUrl = root;
+ this.servicePath = path;
+ }
+ }
+
+ private static ApiComponents apiComponentsFromUrl(String urlString) {
+ try {
+ URL url = new URL(urlString);
+ String rootUrl = url.getProtocol() + "://" + url.getHost() +
+ (url.getPort() > 0 ? ":" + url.getPort() : "");
+ return new ApiComponents(rootUrl, url.getPath());
+ } catch (MalformedURLException e) {
+ throw new RuntimeException("Invalid URL: " + urlString);
+ }
+ }
+
+ /**
+ * Returns a Google Cloud Dataflow client builder.
+ */
+ public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options) {
+ String servicePath = options.getDataflowEndpoint();
+ ApiComponents components;
+ if (servicePath.contains("://")) {
+ components = apiComponentsFromUrl(servicePath);
+ } else {
+ components = new ApiComponents(options.getApiRootUrl(), servicePath);
+ }
+
+ return new Dataflow.Builder(getTransport(),
+ getJsonFactory(),
+ chainHttpRequestInitializer(
+ options.getGcpCredential(),
+ // Do not log 404. It clutters the output and is possibly even required by the caller.
+ new RetryHttpRequestInitializer(ImmutableList.of(404))))
+ .setApplicationName(options.getAppName())
+ .setRootUrl(components.rootUrl)
+ .setServicePath(components.servicePath)
+ .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
+ }
+
+ public static Clouddebugger.Builder newClouddebuggerClient(DataflowPipelineOptions options) {
+ return new Clouddebugger.Builder(getTransport(),
+ getJsonFactory(),
+ chainHttpRequestInitializer(options.getGcpCredential(), new RetryHttpRequestInitializer()))
+ .setApplicationName(options.getAppName())
+ .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
+ }
+
+ /**
+ * Returns a Dataflow client that does not automatically retry failed
+ * requests.
+ */
+ public static Dataflow.Builder
+ newRawDataflowClient(DataflowPipelineOptions options) {
+ return newDataflowClient(options)
+ .setHttpRequestInitializer(options.getGcpCredential())
+ .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
+ }
+
+ private static HttpRequestInitializer chainHttpRequestInitializer(
+ Credential credential, HttpRequestInitializer httpRequestInitializer) {
+ if (credential == null) {
+ return httpRequestInitializer;
+ } else {
+ return new ChainingHttpRequestInitializer(credential, httpRequestInitializer);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java
new file mode 100644
index 0000000..8e7cbbe
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java
@@ -0,0 +1,55 @@
+/*
+ * 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.dataflow.util;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.common.base.Preconditions;
+
+import java.util.List;
+
+/**
+ * Utility class for staging files to GCS.
+ */
+public class GcsStager implements Stager {
+ private DataflowPipelineOptions options;
+
+ private GcsStager(DataflowPipelineOptions options) {
+ this.options = options;
+ }
+
+ public static GcsStager fromOptions(PipelineOptions options) {
+ return new GcsStager(options.as(DataflowPipelineOptions.class));
+ }
+
+ @Override
+ public List<DataflowPackage> stageFiles() {
+ Preconditions.checkNotNull(options.getStagingLocation());
+ List<String> filesToStage = options.getFilesToStage();
+ String windmillBinary =
+ options.as(DataflowPipelineDebugOptions.class).getOverrideWindmillBinary();
+ if (windmillBinary != null) {
+ filesToStage.add("windmill_main=" + windmillBinary);
+ }
+ return PackageUtil.stageClasspathElements(
+ options.getFilesToStage(), options.getStagingLocation());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java
new file mode 100644
index 0000000..2eec9cc
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java
@@ -0,0 +1,237 @@
+/*
+ * 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.dataflow.util;
+
+import static org.apache.beam.sdk.util.TimeUtil.fromCloudTime;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.util.TimeUtil;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages;
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.ListJobMessagesResponse;
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.ImmutableMap;
+
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * A helper class for monitoring jobs submitted to the service.
+ */
+public final class MonitoringUtil {
+
+ private static final String GCLOUD_DATAFLOW_PREFIX = "gcloud alpha dataflow";
+ private static final String ENDPOINT_OVERRIDE_ENV_VAR =
+ "CLOUDSDK_API_ENDPOINT_OVERRIDES_DATAFLOW";
+
+ private static final Map<String, State> DATAFLOW_STATE_TO_JOB_STATE =
+ ImmutableMap
+ .<String, State>builder()
+ .put("JOB_STATE_UNKNOWN", State.UNKNOWN)
+ .put("JOB_STATE_STOPPED", State.STOPPED)
+ .put("JOB_STATE_RUNNING", State.RUNNING)
+ .put("JOB_STATE_DONE", State.DONE)
+ .put("JOB_STATE_FAILED", State.FAILED)
+ .put("JOB_STATE_CANCELLED", State.CANCELLED)
+ .put("JOB_STATE_UPDATED", State.UPDATED)
+ .build();
+
+ private String projectId;
+ private Messages messagesClient;
+
+ /**
+ * An interface that can be used for defining callbacks to receive a list
+ * of JobMessages containing monitoring information.
+ */
+ public interface JobMessagesHandler {
+ /** Process the rows. */
+ void process(List<JobMessage> messages);
+ }
+
+ /** A handler that prints monitoring messages to a stream. */
+ public static class PrintHandler implements JobMessagesHandler {
+ private PrintStream out;
+
+ /**
+ * Construct the handler.
+ *
+ * @param stream The stream to write the messages to.
+ */
+ public PrintHandler(PrintStream stream) {
+ out = stream;
+ }
+
+ @Override
+ public void process(List<JobMessage> messages) {
+ for (JobMessage message : messages) {
+ if (message.getMessageText() == null || message.getMessageText().isEmpty()) {
+ continue;
+ }
+ String importanceString = null;
+ if (message.getMessageImportance() == null) {
+ continue;
+ } else if (message.getMessageImportance().equals("JOB_MESSAGE_ERROR")) {
+ importanceString = "Error: ";
+ } else if (message.getMessageImportance().equals("JOB_MESSAGE_WARNING")) {
+ importanceString = "Warning: ";
+ } else if (message.getMessageImportance().equals("JOB_MESSAGE_BASIC")) {
+ importanceString = "Basic: ";
+ } else if (message.getMessageImportance().equals("JOB_MESSAGE_DETAILED")) {
+ importanceString = "Detail: ";
+ } else {
+ // TODO: Remove filtering here once getJobMessages supports minimum
+ // importance.
+ continue;
+ }
+ @Nullable Instant time = TimeUtil.fromCloudTime(message.getTime());
+ if (time == null) {
+ out.print("UNKNOWN TIMESTAMP: ");
+ } else {
+ out.print(time + ": ");
+ }
+ if (importanceString != null) {
+ out.print(importanceString);
+ }
+ out.println(message.getMessageText());
+ }
+ out.flush();
+ }
+ }
+
+ /** Construct a helper for monitoring. */
+ public MonitoringUtil(String projectId, Dataflow dataflow) {
+ this(projectId, dataflow.projects().jobs().messages());
+ }
+
+ // @VisibleForTesting
+ MonitoringUtil(String projectId, Messages messagesClient) {
+ this.projectId = projectId;
+ this.messagesClient = messagesClient;
+ }
+
+ /**
+ * Comparator for sorting rows in increasing order based on timestamp.
+ */
+ public static class TimeStampComparator implements Comparator<JobMessage> {
+ @Override
+ public int compare(JobMessage o1, JobMessage o2) {
+ @Nullable Instant t1 = fromCloudTime(o1.getTime());
+ if (t1 == null) {
+ return -1;
+ }
+ @Nullable Instant t2 = fromCloudTime(o2.getTime());
+ if (t2 == null) {
+ return 1;
+ }
+ return t1.compareTo(t2);
+ }
+ }
+
+ /**
+ * Return job messages sorted in ascending order by timestamp.
+ * @param jobId The id of the job to get the messages for.
+ * @param startTimestampMs Return only those messages with a
+ * timestamp greater than this value.
+ * @return collection of messages
+ * @throws IOException
+ */
+ public ArrayList<JobMessage> getJobMessages(
+ String jobId, long startTimestampMs) throws IOException {
+ // TODO: Allow filtering messages by importance
+ Instant startTimestamp = new Instant(startTimestampMs);
+ ArrayList<JobMessage> allMessages = new ArrayList<>();
+ String pageToken = null;
+ while (true) {
+ Messages.List listRequest = messagesClient.list(projectId, jobId);
+ if (pageToken != null) {
+ listRequest.setPageToken(pageToken);
+ }
+ ListJobMessagesResponse response = listRequest.execute();
+
+ if (response == null || response.getJobMessages() == null) {
+ return allMessages;
+ }
+
+ for (JobMessage m : response.getJobMessages()) {
+ @Nullable Instant timestamp = fromCloudTime(m.getTime());
+ if (timestamp == null) {
+ continue;
+ }
+ if (timestamp.isAfter(startTimestamp)) {
+ allMessages.add(m);
+ }
+ }
+
+ if (response.getNextPageToken() == null) {
+ break;
+ } else {
+ pageToken = response.getNextPageToken();
+ }
+ }
+
+ Collections.sort(allMessages, new TimeStampComparator());
+ return allMessages;
+ }
+
+ public static String getJobMonitoringPageURL(String projectName, String jobId) {
+ try {
+ // Project name is allowed in place of the project id: the user will be redirected to a URL
+ // that has the project name replaced with project id.
+ return String.format(
+ "https://console.developers.google.com/project/%s/dataflow/job/%s",
+ URLEncoder.encode(projectName, "UTF-8"),
+ URLEncoder.encode(jobId, "UTF-8"));
+ } catch (UnsupportedEncodingException e) {
+ // Should never happen.
+ throw new AssertionError("UTF-8 encoding is not supported by the environment", e);
+ }
+ }
+
+ public static String getGcloudCancelCommand(DataflowPipelineOptions options, String jobId) {
+
+ // If using a different Dataflow API than default, prefix command with an API override.
+ String dataflowApiOverridePrefix = "";
+ String apiUrl = options.getDataflowClient().getBaseUrl();
+ if (!apiUrl.equals(Dataflow.DEFAULT_BASE_URL)) {
+ dataflowApiOverridePrefix = String.format("%s=%s ", ENDPOINT_OVERRIDE_ENV_VAR, apiUrl);
+ }
+
+ // Assemble cancel command from optional prefix and project/job parameters.
+ return String.format("%s%s jobs --project=%s cancel %s",
+ dataflowApiOverridePrefix, GCLOUD_DATAFLOW_PREFIX, options.getProject(), jobId);
+ }
+
+ public static State toState(String stateName) {
+ return MoreObjects.firstNonNull(DATAFLOW_STATE_TO_JOB_STATE.get(stateName),
+ State.UNKNOWN);
+ }
+}
[05/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java
deleted file mode 100644
index d496f38..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineJobTest.java
+++ /dev/null
@@ -1,606 +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.runners;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.allOf;
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.empty;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
-import static org.hamcrest.Matchers.hasEntry;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.lessThanOrEqualTo;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.runners.dataflow.DataflowAggregatorTransforms;
-import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
-import org.apache.beam.sdk.util.MonitoringUtil;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get;
-import com.google.api.services.dataflow.Dataflow.Projects.Jobs.GetMetrics;
-import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.JobMetrics;
-import com.google.api.services.dataflow.model.MetricStructuredName;
-import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSetMultimap;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.net.SocketTimeoutException;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Tests for DataflowPipelineJob.
- */
-@RunWith(JUnit4.class)
-public class DataflowPipelineJobTest {
- private static final String PROJECT_ID = "someProject";
- private static final String JOB_ID = "1234";
-
- @Mock
- private Dataflow mockWorkflowClient;
- @Mock
- private Dataflow.Projects mockProjects;
- @Mock
- private Dataflow.Projects.Jobs mockJobs;
- @Rule
- public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper();
-
- @Rule
- public ExpectedException thrown = ExpectedException.none();
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
-
- when(mockWorkflowClient.projects()).thenReturn(mockProjects);
- when(mockProjects.jobs()).thenReturn(mockJobs);
- }
-
- /**
- * Validates that a given time is valid for the total time slept by a
- * AttemptBoundedExponentialBackOff given the number of retries and
- * an initial polling interval.
- *
- * @param pollingIntervalMillis The initial polling interval given.
- * @param attempts The number of attempts made
- * @param timeSleptMillis The amount of time slept by the clock. This is checked
- * against the valid interval.
- */
- void checkValidInterval(long pollingIntervalMillis, int attempts, long timeSleptMillis) {
- long highSum = 0;
- long lowSum = 0;
- for (int i = 1; i < attempts; i++) {
- double currentInterval =
- pollingIntervalMillis
- * Math.pow(AttemptBoundedExponentialBackOff.DEFAULT_MULTIPLIER, i - 1);
- double offset =
- AttemptBoundedExponentialBackOff.DEFAULT_RANDOMIZATION_FACTOR * currentInterval;
- highSum += Math.round(currentInterval + offset);
- lowSum += Math.round(currentInterval - offset);
- }
- assertThat(timeSleptMillis, allOf(greaterThanOrEqualTo(lowSum), lessThanOrEqualTo(highSum)));
- }
-
- @Test
- public void testWaitToFinishMessagesFail() throws Exception {
- Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
- Job statusResponse = new Job();
- statusResponse.setCurrentState("JOB_STATE_" + State.DONE.name());
- when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
- when(statusRequest.execute()).thenReturn(statusResponse);
-
- MonitoringUtil.JobMessagesHandler jobHandler = mock(MonitoringUtil.JobMessagesHandler.class);
- Dataflow.Projects.Jobs.Messages mockMessages =
- mock(Dataflow.Projects.Jobs.Messages.class);
- Messages.List listRequest = mock(Dataflow.Projects.Jobs.Messages.List.class);
- when(mockJobs.messages()).thenReturn(mockMessages);
- when(mockMessages.list(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(listRequest);
- when(listRequest.execute()).thenThrow(SocketTimeoutException.class);
- DataflowAggregatorTransforms dataflowAggregatorTransforms =
- mock(DataflowAggregatorTransforms.class);
-
- DataflowPipelineJob job = new DataflowPipelineJob(
- PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-
- State state = job.waitToFinish(5, TimeUnit.MINUTES, jobHandler, fastClock, fastClock);
- assertEquals(null, state);
- }
-
- public State mockWaitToFinishInState(State state) throws Exception {
- Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
- Job statusResponse = new Job();
- statusResponse.setCurrentState("JOB_STATE_" + state.name());
-
- when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
- when(statusRequest.execute()).thenReturn(statusResponse);
- DataflowAggregatorTransforms dataflowAggregatorTransforms =
- mock(DataflowAggregatorTransforms.class);
-
- DataflowPipelineJob job = new DataflowPipelineJob(
- PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-
- return job.waitToFinish(1, TimeUnit.MINUTES, null, fastClock, fastClock);
- }
-
- /**
- * Tests that the {@link DataflowPipelineJob} understands that the {@link State#DONE DONE}
- * state is terminal.
- */
- @Test
- public void testWaitToFinishDone() throws Exception {
- assertEquals(State.DONE, mockWaitToFinishInState(State.DONE));
- }
-
- /**
- * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
- * state is terminal.
- */
- @Test
- public void testWaitToFinishFailed() throws Exception {
- assertEquals(State.FAILED, mockWaitToFinishInState(State.FAILED));
- }
-
- /**
- * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
- * state is terminal.
- */
- @Test
- public void testWaitToFinishCancelled() throws Exception {
- assertEquals(State.CANCELLED, mockWaitToFinishInState(State.CANCELLED));
- }
-
- /**
- * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
- * state is terminal.
- */
- @Test
- public void testWaitToFinishUpdated() throws Exception {
- assertEquals(State.UPDATED, mockWaitToFinishInState(State.UPDATED));
- }
-
- @Test
- public void testWaitToFinishFail() throws Exception {
- Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
- when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
- when(statusRequest.execute()).thenThrow(IOException.class);
- DataflowAggregatorTransforms dataflowAggregatorTransforms =
- mock(DataflowAggregatorTransforms.class);
-
- DataflowPipelineJob job = new DataflowPipelineJob(
- PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-
- long startTime = fastClock.nanoTime();
- State state = job.waitToFinish(5, TimeUnit.MINUTES, null, fastClock, fastClock);
- assertEquals(null, state);
- long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
- checkValidInterval(DataflowPipelineJob.MESSAGES_POLLING_INTERVAL,
- DataflowPipelineJob.MESSAGES_POLLING_ATTEMPTS, timeDiff);
- }
-
- @Test
- public void testWaitToFinishTimeFail() throws Exception {
- Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
- when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
- when(statusRequest.execute()).thenThrow(IOException.class);
- DataflowAggregatorTransforms dataflowAggregatorTransforms =
- mock(DataflowAggregatorTransforms.class);
-
- DataflowPipelineJob job = new DataflowPipelineJob(
- PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
- long startTime = fastClock.nanoTime();
- State state = job.waitToFinish(4, TimeUnit.MILLISECONDS, null, fastClock, fastClock);
- assertEquals(null, state);
- long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
- // Should only sleep for the 4 ms remaining.
- assertEquals(timeDiff, 4L);
- }
-
- @Test
- public void testGetStateReturnsServiceState() throws Exception {
- Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
- Job statusResponse = new Job();
- statusResponse.setCurrentState("JOB_STATE_" + State.RUNNING.name());
-
- when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
- when(statusRequest.execute()).thenReturn(statusResponse);
-
- DataflowAggregatorTransforms dataflowAggregatorTransforms =
- mock(DataflowAggregatorTransforms.class);
-
- DataflowPipelineJob job = new DataflowPipelineJob(
- PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-
- assertEquals(
- State.RUNNING,
- job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock));
- }
-
- @Test
- public void testGetStateWithExceptionReturnsUnknown() throws Exception {
- Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
-
- when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
- when(statusRequest.execute()).thenThrow(IOException.class);
- DataflowAggregatorTransforms dataflowAggregatorTransforms =
- mock(DataflowAggregatorTransforms.class);
-
- DataflowPipelineJob job = new DataflowPipelineJob(
- PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
-
- long startTime = fastClock.nanoTime();
- assertEquals(
- State.UNKNOWN,
- job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock));
- long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
- checkValidInterval(DataflowPipelineJob.STATUS_POLLING_INTERVAL,
- DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, timeDiff);
- }
-
- @Test
- public void testGetAggregatorValuesWithNoMetricUpdatesReturnsEmptyValue()
- throws IOException, AggregatorRetrievalException {
- Aggregator<?, ?> aggregator = mock(Aggregator.class);
- @SuppressWarnings("unchecked")
- PTransform<PInput, POutput> pTransform = mock(PTransform.class);
- String stepName = "s1";
- String fullName = "Foo/Bar/Baz";
- AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
- DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
- ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
- ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
-
- GetMetrics getMetrics = mock(GetMetrics.class);
- when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
- JobMetrics jobMetrics = new JobMetrics();
- when(getMetrics.execute()).thenReturn(jobMetrics);
-
- jobMetrics.setMetrics(ImmutableList.<MetricUpdate>of());
-
- Get getState = mock(Get.class);
- when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
- Job modelJob = new Job();
- when(getState.execute()).thenReturn(modelJob);
- modelJob.setCurrentState(State.RUNNING.toString());
-
- DataflowPipelineJob job =
- new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
- AggregatorValues<?> values = job.getAggregatorValues(aggregator);
-
- assertThat(values.getValues(), empty());
- }
-
- @Test
- public void testGetAggregatorValuesWithNullMetricUpdatesReturnsEmptyValue()
- throws IOException, AggregatorRetrievalException {
- Aggregator<?, ?> aggregator = mock(Aggregator.class);
- @SuppressWarnings("unchecked")
- PTransform<PInput, POutput> pTransform = mock(PTransform.class);
- String stepName = "s1";
- String fullName = "Foo/Bar/Baz";
- AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
- DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
- ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
- ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
-
- GetMetrics getMetrics = mock(GetMetrics.class);
- when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
- JobMetrics jobMetrics = new JobMetrics();
- when(getMetrics.execute()).thenReturn(jobMetrics);
-
- jobMetrics.setMetrics(null);
-
- Get getState = mock(Get.class);
- when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
- Job modelJob = new Job();
- when(getState.execute()).thenReturn(modelJob);
- modelJob.setCurrentState(State.RUNNING.toString());
-
- DataflowPipelineJob job =
- new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
- AggregatorValues<?> values = job.getAggregatorValues(aggregator);
-
- assertThat(values.getValues(), empty());
- }
-
- @Test
- public void testGetAggregatorValuesWithSingleMetricUpdateReturnsSingletonCollection()
- throws IOException, AggregatorRetrievalException {
- CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
- String aggregatorName = "agg";
- Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
- @SuppressWarnings("unchecked")
- PTransform<PInput, POutput> pTransform = mock(PTransform.class);
- String stepName = "s1";
- String fullName = "Foo/Bar/Baz";
- AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
- DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
- ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
- ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
-
- GetMetrics getMetrics = mock(GetMetrics.class);
- when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
- JobMetrics jobMetrics = new JobMetrics();
- when(getMetrics.execute()).thenReturn(jobMetrics);
-
- MetricUpdate update = new MetricUpdate();
- long stepValue = 1234L;
- update.setScalar(new BigDecimal(stepValue));
-
- MetricStructuredName structuredName = new MetricStructuredName();
- structuredName.setName(aggregatorName);
- structuredName.setContext(ImmutableMap.of("step", stepName));
- update.setName(structuredName);
-
- jobMetrics.setMetrics(ImmutableList.of(update));
-
- Get getState = mock(Get.class);
- when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
- Job modelJob = new Job();
- when(getState.execute()).thenReturn(modelJob);
- modelJob.setCurrentState(State.RUNNING.toString());
-
- DataflowPipelineJob job =
- new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
- AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
-
- assertThat(values.getValuesAtSteps(), hasEntry(fullName, stepValue));
- assertThat(values.getValuesAtSteps().size(), equalTo(1));
- assertThat(values.getValues(), contains(stepValue));
- assertThat(values.getTotalValue(combineFn), equalTo(Long.valueOf(stepValue)));
- }
-
- @Test
- public void testGetAggregatorValuesWithMultipleMetricUpdatesReturnsCollection()
- throws IOException, AggregatorRetrievalException {
- CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
- String aggregatorName = "agg";
- Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
-
- @SuppressWarnings("unchecked")
- PTransform<PInput, POutput> pTransform = mock(PTransform.class);
- String stepName = "s1";
- String fullName = "Foo/Bar/Baz";
- AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
- @SuppressWarnings("unchecked")
- PTransform<PInput, POutput> otherTransform = mock(PTransform.class);
- String otherStepName = "s88";
- String otherFullName = "Spam/Ham/Eggs";
- AppliedPTransform<?, ?, ?> otherAppliedTransform =
- appliedPTransform(otherFullName, otherTransform);
-
- DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
- ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(
- aggregator, pTransform, aggregator, otherTransform).asMap(),
- ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(
- appliedTransform, stepName, otherAppliedTransform, otherStepName));
-
- GetMetrics getMetrics = mock(GetMetrics.class);
- when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
- JobMetrics jobMetrics = new JobMetrics();
- when(getMetrics.execute()).thenReturn(jobMetrics);
-
- MetricUpdate updateOne = new MetricUpdate();
- long stepValue = 1234L;
- updateOne.setScalar(new BigDecimal(stepValue));
-
- MetricStructuredName structuredNameOne = new MetricStructuredName();
- structuredNameOne.setName(aggregatorName);
- structuredNameOne.setContext(ImmutableMap.of("step", stepName));
- updateOne.setName(structuredNameOne);
-
- MetricUpdate updateTwo = new MetricUpdate();
- long stepValueTwo = 1024L;
- updateTwo.setScalar(new BigDecimal(stepValueTwo));
-
- MetricStructuredName structuredNameTwo = new MetricStructuredName();
- structuredNameTwo.setName(aggregatorName);
- structuredNameTwo.setContext(ImmutableMap.of("step", otherStepName));
- updateTwo.setName(structuredNameTwo);
-
- jobMetrics.setMetrics(ImmutableList.of(updateOne, updateTwo));
-
- Get getState = mock(Get.class);
- when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
- Job modelJob = new Job();
- when(getState.execute()).thenReturn(modelJob);
- modelJob.setCurrentState(State.RUNNING.toString());
-
- DataflowPipelineJob job =
- new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
- AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
-
- assertThat(values.getValuesAtSteps(), hasEntry(fullName, stepValue));
- assertThat(values.getValuesAtSteps(), hasEntry(otherFullName, stepValueTwo));
- assertThat(values.getValuesAtSteps().size(), equalTo(2));
- assertThat(values.getValues(), containsInAnyOrder(stepValue, stepValueTwo));
- assertThat(values.getTotalValue(combineFn), equalTo(Long.valueOf(stepValue + stepValueTwo)));
- }
-
- @Test
- public void testGetAggregatorValuesWithUnrelatedMetricUpdateIgnoresUpdate()
- throws IOException, AggregatorRetrievalException {
- CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
- String aggregatorName = "agg";
- Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
- @SuppressWarnings("unchecked")
- PTransform<PInput, POutput> pTransform = mock(PTransform.class);
- String stepName = "s1";
- String fullName = "Foo/Bar/Baz";
- AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
- DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
- ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
- ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
-
- GetMetrics getMetrics = mock(GetMetrics.class);
- when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
- JobMetrics jobMetrics = new JobMetrics();
- when(getMetrics.execute()).thenReturn(jobMetrics);
-
- MetricUpdate ignoredUpdate = new MetricUpdate();
- ignoredUpdate.setScalar(null);
-
- MetricStructuredName ignoredName = new MetricStructuredName();
- ignoredName.setName("ignoredAggregator.elementCount.out0");
- ignoredName.setContext(null);
- ignoredUpdate.setName(ignoredName);
-
- jobMetrics.setMetrics(ImmutableList.of(ignoredUpdate));
-
- Get getState = mock(Get.class);
- when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
- Job modelJob = new Job();
- when(getState.execute()).thenReturn(modelJob);
- modelJob.setCurrentState(State.RUNNING.toString());
-
- DataflowPipelineJob job =
- new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
- AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
-
- assertThat(values.getValuesAtSteps().entrySet(), empty());
- assertThat(values.getValues(), empty());
- }
-
- @Test
- public void testGetAggregatorValuesWithUnusedAggregatorThrowsException()
- throws AggregatorRetrievalException {
- Aggregator<?, ?> aggregator = mock(Aggregator.class);
-
- DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
- ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of().asMap(),
- ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of());
-
- DataflowPipelineJob job =
- new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
- thrown.expect(IllegalArgumentException.class);
- thrown.expectMessage("not used in this pipeline");
-
- job.getAggregatorValues(aggregator);
- }
-
- @Test
- public void testGetAggregatorValuesWhenClientThrowsExceptionThrowsAggregatorRetrievalException()
- throws IOException, AggregatorRetrievalException {
- CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
- String aggregatorName = "agg";
- Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
- @SuppressWarnings("unchecked")
- PTransform<PInput, POutput> pTransform = mock(PTransform.class);
- String stepName = "s1";
- String fullName = "Foo/Bar/Baz";
- AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
-
- DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
- ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
- ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
-
- GetMetrics getMetrics = mock(GetMetrics.class);
- when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
- IOException cause = new IOException();
- when(getMetrics.execute()).thenThrow(cause);
-
- Get getState = mock(Get.class);
- when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
- Job modelJob = new Job();
- when(getState.execute()).thenReturn(modelJob);
- modelJob.setCurrentState(State.RUNNING.toString());
-
- DataflowPipelineJob job =
- new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
-
- thrown.expect(AggregatorRetrievalException.class);
- thrown.expectCause(is(cause));
- thrown.expectMessage(aggregator.toString());
- thrown.expectMessage("when retrieving Aggregator values for");
-
- job.getAggregatorValues(aggregator);
- }
-
- private static class TestAggregator<InT, OutT> implements Aggregator<InT, OutT> {
- private final CombineFn<InT, ?, OutT> combineFn;
- private final String name;
-
- public TestAggregator(CombineFn<InT, ?, OutT> combineFn, String name) {
- this.combineFn = combineFn;
- this.name = name;
- }
-
- @Override
- public void addValue(InT value) {
- throw new AssertionError();
- }
-
- @Override
- public String getName() {
- return name;
- }
-
- @Override
- public CombineFn<InT, ?, OutT> getCombineFn() {
- return combineFn;
- }
- }
-
- private AppliedPTransform<?, ?, ?> appliedPTransform(
- String fullName, PTransform<PInput, POutput> transform) {
- return AppliedPTransform.of(fullName, mock(PInput.class), mock(POutput.class), transform);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java
deleted file mode 100644
index 67ed2fd..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrarTest.java
+++ /dev/null
@@ -1,74 +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.runners;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import org.apache.beam.sdk.options.BlockingDataflowPipelineOptions;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.util.ServiceLoader;
-
-/** Tests for {@link DataflowPipelineRegistrar}. */
-@RunWith(JUnit4.class)
-public class DataflowPipelineRegistrarTest {
- @Test
- public void testCorrectOptionsAreReturned() {
- assertEquals(ImmutableList.of(DataflowPipelineOptions.class,
- BlockingDataflowPipelineOptions.class),
- new DataflowPipelineRegistrar.Options().getPipelineOptions());
- }
-
- @Test
- public void testCorrectRunnersAreReturned() {
- assertEquals(ImmutableList.of(DataflowPipelineRunner.class,
- BlockingDataflowPipelineRunner.class),
- new DataflowPipelineRegistrar.Runner().getPipelineRunners());
- }
-
- @Test
- public void testServiceLoaderForOptions() {
- for (PipelineOptionsRegistrar registrar :
- Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
- if (registrar instanceof DataflowPipelineRegistrar.Options) {
- return;
- }
- }
- fail("Expected to find " + DataflowPipelineRegistrar.Options.class);
- }
-
- @Test
- public void testServiceLoaderForRunner() {
- for (PipelineRunnerRegistrar registrar :
- Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
- if (registrar instanceof DataflowPipelineRegistrar.Runner) {
- return;
- }
- }
- fail("Expected to find " + DataflowPipelineRegistrar.Runner.class);
- }
-}
[18/21] incubator-beam git commit: Update pom.xml for java8tests
Posted by dh...@apache.org.
Update pom.xml for java8tests
Java8tests module doesn't have sources, only tests. Hence, all dependencies
should have scope of test. If not, dependency analysis correctly finds unused
dependencies.
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/46f74471
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/46f74471
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/46f74471
Branch: refs/heads/master
Commit: 46f744714aae4cd45ba6284e2c9414e4bba5cd3e
Parents: 5e3d7ad
Author: Davor Bonaci <da...@google.com>
Authored: Mon Apr 25 14:16:19 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Apr 26 17:59:39 2016 -0700
----------------------------------------------------------------------
sdks/java/java8tests/pom.xml | 3 +++
1 file changed, 3 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/46f74471/sdks/java/java8tests/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml
index cd7174a..f750a1c 100644
--- a/sdks/java/java8tests/pom.xml
+++ b/sdks/java/java8tests/pom.xml
@@ -150,18 +150,21 @@
<groupId>org.apache.beam</groupId>
<artifactId>java-sdk-all</artifactId>
<version>${project.version}</version>
+ <scope>test</scope>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<version>${guava.version}</version>
+ <scope>test</scope>
</dependency>
<dependency>
<groupId>joda-time</groupId>
<artifactId>joda-time</artifactId>
<version>${joda.version}</version>
+ <scope>test</scope>
</dependency>
<dependency>
[12/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java
deleted file mode 100644
index 428a6ed..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineJob.java
+++ /dev/null
@@ -1,395 +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.runners;
-
-import static org.apache.beam.sdk.util.TimeUtil.fromCloudTime;
-
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.runners.dataflow.DataflowAggregatorTransforms;
-import org.apache.beam.sdk.runners.dataflow.DataflowMetricUpdateExtractor;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
-import org.apache.beam.sdk.util.MapAggregatorValues;
-import org.apache.beam.sdk.util.MonitoringUtil;
-
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.NanoClock;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.JobMessage;
-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;
-
-import java.io.IOException;
-import java.net.SocketTimeoutException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import javax.annotation.Nullable;
-
-/**
- * A DataflowPipelineJob represents a job submitted to Dataflow using
- * {@link DataflowPipelineRunner}.
- */
-public class DataflowPipelineJob implements PipelineResult {
- private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineJob.class);
-
- /**
- * The id for the job.
- */
- private String jobId;
-
- /**
- * Google cloud project to associate this pipeline with.
- */
- private String projectId;
-
- /**
- * Client for the Dataflow service. This can be used to query the service
- * for information about the job.
- */
- private Dataflow dataflowClient;
-
- /**
- * The state the job terminated in or {@code null} if the job has not terminated.
- */
- @Nullable
- private State terminalState = null;
-
- /**
- * The job that replaced this one or {@code null} if the job has not been replaced.
- */
- @Nullable
- private DataflowPipelineJob replacedByJob = null;
-
- private DataflowAggregatorTransforms aggregatorTransforms;
-
- /**
- * The Metric Updates retrieved after the job was in a terminal state.
- */
- private List<MetricUpdate> terminalMetricUpdates;
-
- /**
- * The polling interval for job status and messages information.
- */
- static final long MESSAGES_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2);
- static final long STATUS_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2);
-
- /**
- * The amount of polling attempts for job status and messages information.
- */
- static final int MESSAGES_POLLING_ATTEMPTS = 10;
- static final int STATUS_POLLING_ATTEMPTS = 5;
-
- /**
- * Constructs the job.
- *
- * @param projectId the project id
- * @param jobId the job id
- * @param dataflowClient the client for the Dataflow Service
- */
- public DataflowPipelineJob(String projectId, String jobId, Dataflow dataflowClient,
- DataflowAggregatorTransforms aggregatorTransforms) {
- this.projectId = projectId;
- this.jobId = jobId;
- this.dataflowClient = dataflowClient;
- this.aggregatorTransforms = aggregatorTransforms;
- }
-
- /**
- * Get the id of this job.
- */
- public String getJobId() {
- return jobId;
- }
-
- /**
- * Get the project this job exists in.
- */
- public String getProjectId() {
- return projectId;
- }
-
- /**
- * Returns a new {@link DataflowPipelineJob} for the job that replaced this one, if applicable.
- *
- * @throws IllegalStateException if called before the job has terminated or if the job terminated
- * but was not updated
- */
- public DataflowPipelineJob getReplacedByJob() {
- if (terminalState == null) {
- throw new IllegalStateException("getReplacedByJob() called before job terminated");
- }
- if (replacedByJob == null) {
- throw new IllegalStateException("getReplacedByJob() called for job that was not replaced");
- }
- return replacedByJob;
- }
-
- /**
- * Get the Cloud Dataflow API Client used by this job.
- */
- public Dataflow getDataflowClient() {
- return dataflowClient;
- }
-
- /**
- * Waits for the job to finish and return the final status.
- *
- * @param timeToWait The time to wait in units timeUnit for the job to finish.
- * Provide a value less than 1 ms for an infinite wait.
- * @param timeUnit The unit of time for timeToWait.
- * @param messageHandler If non null this handler will be invoked for each
- * batch of messages received.
- * @return The final state of the job or null on timeout or if the
- * thread is interrupted.
- * @throws IOException If there is a persistent problem getting job
- * information.
- * @throws InterruptedException
- */
- @Nullable
- public State waitToFinish(
- long timeToWait,
- TimeUnit timeUnit,
- MonitoringUtil.JobMessagesHandler messageHandler)
- throws IOException, InterruptedException {
- return waitToFinish(timeToWait, timeUnit, messageHandler, Sleeper.DEFAULT, NanoClock.SYSTEM);
- }
-
- /**
- * Wait for the job to finish and return the final status.
- *
- * @param timeToWait The time to wait in units timeUnit for the job to finish.
- * Provide a value less than 1 ms for an infinite wait.
- * @param timeUnit The unit of time for timeToWait.
- * @param messageHandler If non null this handler will be invoked for each
- * batch of messages received.
- * @param sleeper A sleeper to use to sleep between attempts.
- * @param nanoClock A nanoClock used to time the total time taken.
- * @return The final state of the job or null on timeout or if the
- * thread is interrupted.
- * @throws IOException If there is a persistent problem getting job
- * information.
- * @throws InterruptedException
- */
- @Nullable
- @VisibleForTesting
- State waitToFinish(
- long timeToWait,
- TimeUnit timeUnit,
- MonitoringUtil.JobMessagesHandler messageHandler,
- Sleeper sleeper,
- NanoClock nanoClock)
- throws IOException, InterruptedException {
- MonitoringUtil monitor = new MonitoringUtil(projectId, dataflowClient);
-
- long lastTimestamp = 0;
- BackOff backoff =
- timeUnit.toMillis(timeToWait) > 0
- ? new AttemptAndTimeBoundedExponentialBackOff(
- MESSAGES_POLLING_ATTEMPTS,
- MESSAGES_POLLING_INTERVAL,
- timeUnit.toMillis(timeToWait),
- AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS,
- nanoClock)
- : new AttemptBoundedExponentialBackOff(
- MESSAGES_POLLING_ATTEMPTS, MESSAGES_POLLING_INTERVAL);
- State state;
- do {
- // Get the state of the job before listing messages. This ensures we always fetch job
- // messages after the job finishes to ensure we have all them.
- state = getStateWithRetries(1, sleeper);
- boolean hasError = state == State.UNKNOWN;
-
- if (messageHandler != null && !hasError) {
- // Process all the job messages that have accumulated so far.
- try {
- List<JobMessage> allMessages = monitor.getJobMessages(
- jobId, lastTimestamp);
-
- if (!allMessages.isEmpty()) {
- lastTimestamp =
- fromCloudTime(allMessages.get(allMessages.size() - 1).getTime()).getMillis();
- messageHandler.process(allMessages);
- }
- } catch (GoogleJsonResponseException | SocketTimeoutException e) {
- hasError = true;
- LOG.warn("There were problems getting current job messages: {}.", e.getMessage());
- LOG.debug("Exception information:", e);
- }
- }
-
- if (!hasError) {
- backoff.reset();
- // Check if the job is done.
- if (state.isTerminal()) {
- return state;
- }
- }
- } while(BackOffUtils.next(sleeper, backoff));
- LOG.warn("No terminal state was returned. State value {}", state);
- return null; // Timed out.
- }
-
- /**
- * Cancels the job.
- * @throws IOException if there is a problem executing the cancel request.
- */
- public void cancel() throws IOException {
- Job content = new Job();
- content.setProjectId(projectId);
- content.setId(jobId);
- content.setRequestedState("JOB_STATE_CANCELLED");
- dataflowClient.projects().jobs()
- .update(projectId, jobId, content)
- .execute();
- }
-
- @Override
- public State getState() {
- if (terminalState != null) {
- return terminalState;
- }
-
- return getStateWithRetries(STATUS_POLLING_ATTEMPTS, Sleeper.DEFAULT);
- }
-
- /**
- * Attempts to get the state. Uses exponential backoff on failure up to the maximum number
- * of passed in attempts.
- *
- * @param attempts The amount of attempts to make.
- * @param sleeper Object used to do the sleeps between attempts.
- * @return The state of the job or State.UNKNOWN in case of failure.
- */
- @VisibleForTesting
- State getStateWithRetries(int attempts, Sleeper sleeper) {
- if (terminalState != null) {
- return terminalState;
- }
- try {
- Job job = getJobWithRetries(attempts, sleeper);
- return MonitoringUtil.toState(job.getCurrentState());
- } catch (IOException exn) {
- // The only IOException that getJobWithRetries is permitted to throw is the final IOException
- // that caused the failure of retry. Other exceptions are wrapped in an unchecked exceptions
- // and will propagate.
- return State.UNKNOWN;
- }
- }
-
- /**
- * Attempts to get the underlying {@link Job}. Uses exponential backoff on failure up to the
- * maximum number of passed in attempts.
- *
- * @param attempts The amount of attempts to make.
- * @param sleeper Object used to do the sleeps between attempts.
- * @return The underlying {@link Job} object.
- * @throws IOException When the maximum number of retries is exhausted, the last exception is
- * thrown.
- */
- @VisibleForTesting
- Job getJobWithRetries(int attempts, Sleeper sleeper) throws IOException {
- AttemptBoundedExponentialBackOff backoff =
- new AttemptBoundedExponentialBackOff(attempts, STATUS_POLLING_INTERVAL);
-
- // Retry loop ends in return or throw
- while (true) {
- try {
- Job job = dataflowClient
- .projects()
- .jobs()
- .get(projectId, jobId)
- .execute();
- State currentState = MonitoringUtil.toState(job.getCurrentState());
- if (currentState.isTerminal()) {
- terminalState = currentState;
- replacedByJob = new DataflowPipelineJob(
- getProjectId(), job.getReplacedByJobId(), dataflowClient, aggregatorTransforms);
- }
- return job;
- } catch (IOException exn) {
- LOG.warn("There were problems getting current job status: {}.", exn.getMessage());
- LOG.debug("Exception information:", exn);
-
- if (!nextBackOff(sleeper, backoff)) {
- throw exn;
- }
- }
- }
- }
-
- /**
- * Identical to {@link BackOffUtils#next} but without checked exceptions.
- */
- private boolean nextBackOff(Sleeper sleeper, BackOff backoff) {
- try {
- return BackOffUtils.next(sleeper, backoff);
- } catch (InterruptedException | IOException e) {
- if (e instanceof InterruptedException) {
- Thread.currentThread().interrupt();
- }
- throw Throwables.propagate(e);
- }
- }
-
- @Override
- public <OutputT> AggregatorValues<OutputT> getAggregatorValues(Aggregator<?, OutputT> aggregator)
- throws AggregatorRetrievalException {
- try {
- return new MapAggregatorValues<>(fromMetricUpdates(aggregator));
- } catch (IOException e) {
- throw new AggregatorRetrievalException(
- "IOException when retrieving Aggregator values for Aggregator " + aggregator, e);
- }
- }
-
- private <OutputT> Map<String, OutputT> fromMetricUpdates(Aggregator<?, OutputT> aggregator)
- throws IOException {
- if (aggregatorTransforms.contains(aggregator)) {
- List<MetricUpdate> metricUpdates;
- if (terminalMetricUpdates != null) {
- metricUpdates = terminalMetricUpdates;
- } else {
- boolean terminal = getState().isTerminal();
- JobMetrics jobMetrics =
- dataflowClient.projects().jobs().getMetrics(projectId, jobId).execute();
- metricUpdates = jobMetrics.getMetrics();
- if (terminal && jobMetrics.getMetrics() != null) {
- terminalMetricUpdates = metricUpdates;
- }
- }
-
- return DataflowMetricUpdateExtractor.fromMetricUpdates(
- aggregator, aggregatorTransforms, metricUpdates);
- } else {
- throw new IllegalArgumentException(
- "Aggregator " + aggregator + " is not used in this pipeline");
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java
deleted file mode 100644
index b0f72ed..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRegistrar.java
+++ /dev/null
@@ -1,60 +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.runners;
-
-import org.apache.beam.sdk.options.BlockingDataflowPipelineOptions;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-
-import com.google.auto.service.AutoService;
-import com.google.common.collect.ImmutableList;
-
-/**
- * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for the
- * {@link DataflowPipelineRunner}.
- */
-public class DataflowPipelineRegistrar {
- private DataflowPipelineRegistrar() { }
-
- /**
- * Register the {@link DataflowPipelineOptions} and {@link BlockingDataflowPipelineOptions}.
- */
- @AutoService(PipelineOptionsRegistrar.class)
- public static class Options implements PipelineOptionsRegistrar {
- @Override
- public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
- return ImmutableList.<Class<? extends PipelineOptions>>of(
- DataflowPipelineOptions.class,
- BlockingDataflowPipelineOptions.class);
- }
- }
-
- /**
- * Register the {@link DataflowPipelineRunner} and {@link BlockingDataflowPipelineRunner}.
- */
- @AutoService(PipelineRunnerRegistrar.class)
- public static class Runner implements PipelineRunnerRegistrar {
- @Override
- public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
- return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
- DataflowPipelineRunner.class,
- BlockingDataflowPipelineRunner.class);
- }
- }
-}
[13/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java
new file mode 100644
index 0000000..cff7e2b
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java
@@ -0,0 +1,333 @@
+/*
+ * 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.dataflow.util;
+
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.MimeTypes;
+import org.apache.beam.sdk.util.ZipFiles;
+
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.cloud.hadoop.util.ApiErrorExtractor;
+import com.google.common.hash.Funnels;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import com.google.common.io.CountingOutputStream;
+import com.google.common.io.Files;
+
+import com.fasterxml.jackson.core.Base64Variants;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+
+/** Helper routines for packages. */
+public class PackageUtil {
+ private static final Logger LOG = LoggerFactory.getLogger(PackageUtil.class);
+ /**
+ * A reasonable upper bound on the number of jars required to launch a Dataflow job.
+ */
+ public static final int SANE_CLASSPATH_SIZE = 1000;
+ /**
+ * The initial interval to use between package staging attempts.
+ */
+ private static final long INITIAL_BACKOFF_INTERVAL_MS = 5000L;
+ /**
+ * The maximum number of attempts when staging a file.
+ */
+ private static final int MAX_ATTEMPTS = 5;
+
+ /**
+ * Translates exceptions from API calls.
+ */
+ private static final ApiErrorExtractor ERROR_EXTRACTOR = new ApiErrorExtractor();
+
+ /**
+ * Creates a DataflowPackage containing information about how a classpath element should be
+ * staged, including the staging destination as well as its size and hash.
+ *
+ * @param classpathElement The local path for the classpath element.
+ * @param stagingPath The base location for staged classpath elements.
+ * @param overridePackageName If non-null, use the given value as the package name
+ * instead of generating one automatically.
+ * @return The package.
+ */
+ @Deprecated
+ public static DataflowPackage createPackage(File classpathElement,
+ String stagingPath, String overridePackageName) {
+ return createPackageAttributes(classpathElement, stagingPath, overridePackageName)
+ .getDataflowPackage();
+ }
+
+ /**
+ * Compute and cache the attributes of a classpath element that we will need to stage it.
+ *
+ * @param classpathElement the file or directory to be staged.
+ * @param stagingPath The base location for staged classpath elements.
+ * @param overridePackageName If non-null, use the given value as the package name
+ * instead of generating one automatically.
+ * @return a {@link PackageAttributes} that containing metadata about the object to be staged.
+ */
+ static PackageAttributes createPackageAttributes(File classpathElement,
+ String stagingPath, String overridePackageName) {
+ try {
+ boolean directory = classpathElement.isDirectory();
+
+ // Compute size and hash in one pass over file or directory.
+ Hasher hasher = Hashing.md5().newHasher();
+ OutputStream hashStream = Funnels.asOutputStream(hasher);
+ CountingOutputStream countingOutputStream = new CountingOutputStream(hashStream);
+
+ if (!directory) {
+ // Files are staged as-is.
+ Files.asByteSource(classpathElement).copyTo(countingOutputStream);
+ } else {
+ // Directories are recursively zipped.
+ ZipFiles.zipDirectory(classpathElement, countingOutputStream);
+ }
+
+ long size = countingOutputStream.getCount();
+ String hash = Base64Variants.MODIFIED_FOR_URL.encode(hasher.hash().asBytes());
+
+ // Create the DataflowPackage with staging name and location.
+ String uniqueName = getUniqueContentName(classpathElement, hash);
+ String resourcePath = IOChannelUtils.resolve(stagingPath, uniqueName);
+ DataflowPackage target = new DataflowPackage();
+ target.setName(overridePackageName != null ? overridePackageName : uniqueName);
+ target.setLocation(resourcePath);
+
+ return new PackageAttributes(size, hash, directory, target);
+ } catch (IOException e) {
+ throw new RuntimeException("Package setup failure for " + classpathElement, e);
+ }
+ }
+
+ /**
+ * Transfers the classpath elements to the staging location.
+ *
+ * @param classpathElements The elements to stage.
+ * @param stagingPath The base location to stage the elements to.
+ * @return A list of cloud workflow packages, each representing a classpath element.
+ */
+ public static List<DataflowPackage> stageClasspathElements(
+ Collection<String> classpathElements, String stagingPath) {
+ return stageClasspathElements(classpathElements, stagingPath, Sleeper.DEFAULT);
+ }
+
+ // Visible for testing.
+ static List<DataflowPackage> stageClasspathElements(
+ Collection<String> classpathElements, String stagingPath,
+ Sleeper retrySleeper) {
+ LOG.info("Uploading {} files from PipelineOptions.filesToStage to staging location to "
+ + "prepare for execution.", classpathElements.size());
+
+ if (classpathElements.size() > SANE_CLASSPATH_SIZE) {
+ LOG.warn("Your classpath contains {} elements, which Google Cloud Dataflow automatically "
+ + "copies to all workers. Having this many entries on your classpath may be indicative "
+ + "of an issue in your pipeline. You may want to consider trimming the classpath to "
+ + "necessary dependencies only, using --filesToStage pipeline option to override "
+ + "what files are being staged, or bundling several dependencies into one.",
+ classpathElements.size());
+ }
+
+ ArrayList<DataflowPackage> packages = new ArrayList<>();
+
+ if (stagingPath == null) {
+ throw new IllegalArgumentException(
+ "Can't stage classpath elements on because no staging location has been provided");
+ }
+
+ int numUploaded = 0;
+ int numCached = 0;
+ for (String classpathElement : classpathElements) {
+ String packageName = null;
+ if (classpathElement.contains("=")) {
+ String[] components = classpathElement.split("=", 2);
+ packageName = components[0];
+ classpathElement = components[1];
+ }
+
+ File file = new File(classpathElement);
+ if (!file.exists()) {
+ LOG.warn("Skipping non-existent classpath element {} that was specified.",
+ classpathElement);
+ continue;
+ }
+
+ PackageAttributes attributes = createPackageAttributes(file, stagingPath, packageName);
+
+ DataflowPackage workflowPackage = attributes.getDataflowPackage();
+ packages.add(workflowPackage);
+ String target = workflowPackage.getLocation();
+
+ // TODO: Should we attempt to detect the Mime type rather than
+ // always using MimeTypes.BINARY?
+ try {
+ try {
+ long remoteLength = IOChannelUtils.getSizeBytes(target);
+ if (remoteLength == attributes.getSize()) {
+ LOG.debug("Skipping classpath element already staged: {} at {}",
+ classpathElement, target);
+ numCached++;
+ continue;
+ }
+ } catch (FileNotFoundException expected) {
+ // If the file doesn't exist, it means we need to upload it.
+ }
+
+ // Upload file, retrying on failure.
+ AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff(
+ MAX_ATTEMPTS,
+ INITIAL_BACKOFF_INTERVAL_MS);
+ while (true) {
+ try {
+ LOG.debug("Uploading classpath element {} to {}", classpathElement, target);
+ try (WritableByteChannel writer = IOChannelUtils.create(target, MimeTypes.BINARY)) {
+ copyContent(classpathElement, writer);
+ }
+ numUploaded++;
+ break;
+ } catch (IOException e) {
+ if (ERROR_EXTRACTOR.accessDenied(e)) {
+ String errorMessage = String.format(
+ "Uploaded failed due to permissions error, will NOT retry staging "
+ + "of classpath %s. Please verify credentials are valid and that you have "
+ + "write access to %s. Stale credentials can be resolved by executing "
+ + "'gcloud auth login'.", classpathElement, target);
+ LOG.error(errorMessage);
+ throw new IOException(errorMessage, e);
+ } else if (!backoff.atMaxAttempts()) {
+ LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}",
+ classpathElement, e);
+ BackOffUtils.next(retrySleeper, backoff);
+ } else {
+ // Rethrow last error, to be included as a cause in the catch below.
+ LOG.error("Upload failed, will NOT retry staging of classpath: {}",
+ classpathElement, e);
+ throw e;
+ }
+ }
+ }
+ } catch (Exception e) {
+ throw new RuntimeException("Could not stage classpath element: " + classpathElement, e);
+ }
+ }
+
+ LOG.info("Uploading PipelineOptions.filesToStage complete: {} files newly uploaded, "
+ + "{} files cached",
+ numUploaded, numCached);
+
+ return packages;
+ }
+
+ /**
+ * Returns a unique name for a file with a given content hash.
+ *
+ * <p>Directory paths are removed. Example:
+ * <pre>
+ * dir="a/b/c/d", contentHash="f000" => d-f000.jar
+ * file="a/b/c/d.txt", contentHash="f000" => d-f000.txt
+ * file="a/b/c/d", contentHash="f000" => d-f000
+ * </pre>
+ */
+ static String getUniqueContentName(File classpathElement, String contentHash) {
+ String fileName = Files.getNameWithoutExtension(classpathElement.getAbsolutePath());
+ String fileExtension = Files.getFileExtension(classpathElement.getAbsolutePath());
+ if (classpathElement.isDirectory()) {
+ return fileName + "-" + contentHash + ".jar";
+ } else if (fileExtension.isEmpty()) {
+ return fileName + "-" + contentHash;
+ }
+ return fileName + "-" + contentHash + "." + fileExtension;
+ }
+
+ /**
+ * Copies the contents of the classpathElement to the output channel.
+ *
+ * <p>If the classpathElement is a directory, a Zip stream is constructed on the fly,
+ * otherwise the file contents are copied as-is.
+ *
+ * <p>The output channel is not closed.
+ */
+ private static void copyContent(String classpathElement, WritableByteChannel outputChannel)
+ throws IOException {
+ final File classpathElementFile = new File(classpathElement);
+ if (classpathElementFile.isDirectory()) {
+ ZipFiles.zipDirectory(classpathElementFile, Channels.newOutputStream(outputChannel));
+ } else {
+ Files.asByteSource(classpathElementFile).copyTo(Channels.newOutputStream(outputChannel));
+ }
+ }
+ /**
+ * Holds the metadata necessary to stage a file or confirm that a staged file has not changed.
+ */
+ static class PackageAttributes {
+ private final boolean directory;
+ private final long size;
+ private final String hash;
+ private DataflowPackage dataflowPackage;
+
+ public PackageAttributes(long size, String hash, boolean directory,
+ DataflowPackage dataflowPackage) {
+ this.size = size;
+ this.hash = Objects.requireNonNull(hash, "hash");
+ this.directory = directory;
+ this.dataflowPackage = Objects.requireNonNull(dataflowPackage, "dataflowPackage");
+ }
+
+ /**
+ * @return the dataflowPackage
+ */
+ public DataflowPackage getDataflowPackage() {
+ return dataflowPackage;
+ }
+
+ /**
+ * @return the directory
+ */
+ public boolean isDirectory() {
+ return directory;
+ }
+
+ /**
+ * @return the size
+ */
+ public long getSize() {
+ return size;
+ }
+
+ /**
+ * @return the hash
+ */
+ public String getHash() {
+ return hash;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java
new file mode 100644
index 0000000..2172eb4
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java
@@ -0,0 +1,30 @@
+/*
+ * 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.dataflow.util;
+
+import com.google.api.services.dataflow.model.DataflowPackage;
+
+import java.util.List;
+
+/**
+ * Interface for staging files needed for running a Dataflow pipeline.
+ */
+public interface Stager {
+ /* Stage files and return a list of packages. */
+ public List<DataflowPackage> stageFiles();
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java
deleted file mode 100644
index 5b52d30..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/BlockingDataflowPipelineOptions.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.options;
-
-import org.apache.beam.sdk.runners.BlockingDataflowPipelineRunner;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import java.io.PrintStream;
-
-/**
- * Options that are used to configure the {@link BlockingDataflowPipelineRunner}.
- */
-@Description("Configure options on the BlockingDataflowPipelineRunner.")
-public interface BlockingDataflowPipelineOptions extends DataflowPipelineOptions {
- /**
- * Output stream for job status messages.
- */
- @Description("Where messages generated during execution of the Dataflow job will be output.")
- @JsonIgnore
- @Hidden
- @Default.InstanceFactory(StandardOutputFactory.class)
- PrintStream getJobMessageOutput();
- void setJobMessageOutput(PrintStream value);
-
- /**
- * Returns a default of {@link System#out}.
- */
- public static class StandardOutputFactory implements DefaultValueFactory<PrintStream> {
- @Override
- public PrintStream create(PipelineOptions options) {
- return System.out;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java
deleted file mode 100644
index 0464fb4..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/CloudDebuggerOptions.java
+++ /dev/null
@@ -1,53 +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.options;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.api.services.clouddebugger.v2.model.Debuggee;
-
-import javax.annotation.Nullable;
-
-/**
- * Options for controlling Cloud Debugger.
- */
-@Description("[Experimental] Used to configure the Cloud Debugger")
-@Experimental
-@Hidden
-public interface CloudDebuggerOptions {
-
- /** Whether to enable the Cloud Debugger snapshot agent for the current job. */
- @Description("Whether to enable the Cloud Debugger snapshot agent for the current job.")
- boolean getEnableCloudDebugger();
- void setEnableCloudDebugger(boolean enabled);
-
- /** The Cloud Debugger debuggee to associate with. This should not be set directly. */
- @Description("The Cloud Debugger debuggee to associate with. This should not be set directly.")
- @Hidden
- @Nullable Debuggee getDebuggee();
- void setDebuggee(Debuggee debuggee);
-
- /** The maximum cost (as a ratio of CPU time) allowed for evaluating conditional snapshots. */
- @Description(
- "The maximum cost (as a ratio of CPU time) allowed for evaluating conditional snapshots. "
- + "Should be a double between 0 and 1. "
- + "Snapshots will be cancelled if evaluating conditions takes more than this ratio of time.")
- @Default.Double(0.01)
- double getMaxConditionCost();
- void setMaxConditionCost(double maxConditionCost);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java
deleted file mode 100644
index 553f667..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptions.java
+++ /dev/null
@@ -1,242 +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.options;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.util.DataflowPathValidator;
-import org.apache.beam.sdk.util.DataflowTransport;
-import org.apache.beam.sdk.util.GcsStager;
-import org.apache.beam.sdk.util.InstanceBuilder;
-import org.apache.beam.sdk.util.PathValidator;
-import org.apache.beam.sdk.util.Stager;
-
-import com.google.api.services.dataflow.Dataflow;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import java.util.List;
-import java.util.Map;
-
-/**
- * Internal. Options used to control execution of the Dataflow SDK for
- * debugging and testing purposes.
- */
-@Description("[Internal] Options used to control execution of the Dataflow SDK for "
- + "debugging and testing purposes.")
-@Hidden
-public interface DataflowPipelineDebugOptions extends PipelineOptions {
-
- /**
- * The list of backend experiments to enable.
- *
- * <p>Dataflow provides a number of experimental features that can be enabled
- * with this flag.
- *
- * <p>Please sync with the Dataflow team before enabling any experiments.
- */
- @Description("[Experimental] Dataflow provides a number of experimental features that can "
- + "be enabled with this flag. Please sync with the Dataflow team before enabling any "
- + "experiments.")
- @Experimental
- List<String> getExperiments();
- void setExperiments(List<String> value);
-
- /**
- * The root URL for the Dataflow API. {@code dataflowEndpoint} can override this value
- * if it contains an absolute URL, otherwise {@code apiRootUrl} will be combined with
- * {@code dataflowEndpoint} to generate the full URL to communicate with the Dataflow API.
- */
- @Description("The root URL for the Dataflow API. dataflowEndpoint can override this "
- + "value if it contains an absolute URL, otherwise apiRootUrl will be combined with "
- + "dataflowEndpoint to generate the full URL to communicate with the Dataflow API.")
- @Default.String(Dataflow.DEFAULT_ROOT_URL)
- String getApiRootUrl();
- void setApiRootUrl(String value);
-
- /**
- * Dataflow endpoint to use.
- *
- * <p>Defaults to the current version of the Google Cloud Dataflow
- * API, at the time the current SDK version was released.
- *
- * <p>If the string contains "://", then this is treated as a URL,
- * otherwise {@link #getApiRootUrl()} is used as the root
- * URL.
- */
- @Description("The URL for the Dataflow API. If the string contains \"://\", this"
- + " will be treated as the entire URL, otherwise will be treated relative to apiRootUrl.")
- @Default.String(Dataflow.DEFAULT_SERVICE_PATH)
- String getDataflowEndpoint();
- void setDataflowEndpoint(String value);
-
- /**
- * The path to write the translated Dataflow job specification out to
- * at job submission time. The Dataflow job specification will be represented in JSON
- * format.
- */
- @Description("The path to write the translated Dataflow job specification out to "
- + "at job submission time. The Dataflow job specification will be represented in JSON "
- + "format.")
- String getDataflowJobFile();
- void setDataflowJobFile(String value);
-
- /**
- * The class of the validator that should be created and used to validate paths.
- * If pathValidator has not been set explicitly, an instance of this class will be
- * constructed and used as the path validator.
- */
- @Description("The class of the validator that should be created and used to validate paths. "
- + "If pathValidator has not been set explicitly, an instance of this class will be "
- + "constructed and used as the path validator.")
- @Default.Class(DataflowPathValidator.class)
- Class<? extends PathValidator> getPathValidatorClass();
- void setPathValidatorClass(Class<? extends PathValidator> validatorClass);
-
- /**
- * The path validator instance that should be used to validate paths.
- * If no path validator has been set explicitly, the default is to use the instance factory that
- * constructs a path validator based upon the currently set pathValidatorClass.
- */
- @JsonIgnore
- @Description("The path validator instance that should be used to validate paths. "
- + "If no path validator has been set explicitly, the default is to use the instance factory "
- + "that constructs a path validator based upon the currently set pathValidatorClass.")
- @Default.InstanceFactory(PathValidatorFactory.class)
- PathValidator getPathValidator();
- void setPathValidator(PathValidator validator);
-
- /**
- * The class responsible for staging resources to be accessible by workers
- * during job execution. If stager has not been set explicitly, an instance of this class
- * will be created and used as the resource stager.
- */
- @Description("The class of the stager that should be created and used to stage resources. "
- + "If stager has not been set explicitly, an instance of the this class will be created "
- + "and used as the resource stager.")
- @Default.Class(GcsStager.class)
- Class<? extends Stager> getStagerClass();
- void setStagerClass(Class<? extends Stager> stagerClass);
-
- /**
- * The resource stager instance that should be used to stage resources.
- * If no stager has been set explicitly, the default is to use the instance factory
- * that constructs a resource stager based upon the currently set stagerClass.
- */
- @JsonIgnore
- @Description("The resource stager instance that should be used to stage resources. "
- + "If no stager has been set explicitly, the default is to use the instance factory "
- + "that constructs a resource stager based upon the currently set stagerClass.")
- @Default.InstanceFactory(StagerFactory.class)
- Stager getStager();
- void setStager(Stager stager);
-
- /**
- * An instance of the Dataflow client. Defaults to creating a Dataflow client
- * using the current set of options.
- */
- @JsonIgnore
- @Description("An instance of the Dataflow client. Defaults to creating a Dataflow client "
- + "using the current set of options.")
- @Default.InstanceFactory(DataflowClientFactory.class)
- Dataflow getDataflowClient();
- void setDataflowClient(Dataflow value);
-
- /** Returns the default Dataflow client built from the passed in PipelineOptions. */
- public static class DataflowClientFactory implements DefaultValueFactory<Dataflow> {
- @Override
- public Dataflow create(PipelineOptions options) {
- return DataflowTransport.newDataflowClient(
- options.as(DataflowPipelineOptions.class)).build();
- }
- }
-
- /**
- * Mapping of old PTranform names to new ones, specified as JSON
- * <code>{"oldName":"newName",...}</code>. To mark a transform as deleted, make newName the
- * empty string.
- */
- @JsonIgnore
- @Description(
- "Mapping of old PTranform names to new ones, specified as JSON "
- + "{\"oldName\":\"newName\",...}. To mark a transform as deleted, make newName the empty "
- + "string.")
- Map<String, String> getTransformNameMapping();
- void setTransformNameMapping(Map<String, String> value);
-
- /**
- * Custom windmill_main binary to use with the streaming runner.
- */
- @Description("Custom windmill_main binary to use with the streaming runner")
- String getOverrideWindmillBinary();
- void setOverrideWindmillBinary(String value);
-
- /**
- * Number of threads to use on the Dataflow worker harness. If left unspecified,
- * the Dataflow service will compute an appropriate number of threads to use.
- */
- @Description("Number of threads to use on the Dataflow worker harness. If left unspecified, "
- + "the Dataflow service will compute an appropriate number of threads to use.")
- int getNumberOfWorkerHarnessThreads();
- void setNumberOfWorkerHarnessThreads(int value);
-
- /**
- * If {@literal true}, save a heap dump before killing a thread or process which is GC
- * thrashing or out of memory. The location of the heap file will either be echoed back
- * to the user, or the user will be given the opportunity to download the heap file.
- *
- * <p>
- * CAUTION: Heap dumps can of comparable size to the default boot disk. Consider increasing
- * the boot disk size before setting this flag to true.
- */
- @Description("If {@literal true}, save a heap dump before killing a thread or process "
- + "which is GC thrashing or out of memory.")
- boolean getDumpHeapOnOOM();
- void setDumpHeapOnOOM(boolean dumpHeapBeforeExit);
-
- /**
- * Creates a {@link PathValidator} object using the class specified in
- * {@link #getPathValidatorClass()}.
- */
- public static class PathValidatorFactory implements DefaultValueFactory<PathValidator> {
- @Override
- public PathValidator create(PipelineOptions options) {
- DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class);
- return InstanceBuilder.ofType(PathValidator.class)
- .fromClass(debugOptions.getPathValidatorClass())
- .fromFactoryMethod("fromOptions")
- .withArg(PipelineOptions.class, options)
- .build();
- }
- }
-
- /**
- * Creates a {@link Stager} object using the class specified in
- * {@link #getStagerClass()}.
- */
- public static class StagerFactory implements DefaultValueFactory<Stager> {
- @Override
- public Stager create(PipelineOptions options) {
- DataflowPipelineDebugOptions debugOptions = options.as(DataflowPipelineDebugOptions.class);
- return InstanceBuilder.ofType(Stager.class)
- .fromClass(debugOptions.getStagerClass())
- .fromFactoryMethod("fromOptions")
- .withArg(PipelineOptions.class, options)
- .build();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java
deleted file mode 100644
index 7f9d189..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineOptions.java
+++ /dev/null
@@ -1,115 +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.options;
-
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
-
-import com.google.common.base.MoreObjects;
-
-import org.joda.time.DateTimeUtils;
-import org.joda.time.DateTimeZone;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
-
-/**
- * Options that can be used to configure the {@link DataflowPipelineRunner}.
- */
-@Description("Options that configure the Dataflow pipeline.")
-public interface DataflowPipelineOptions
- extends PipelineOptions, GcpOptions, ApplicationNameOptions, DataflowPipelineDebugOptions,
- DataflowPipelineWorkerPoolOptions, BigQueryOptions, GcsOptions, StreamingOptions,
- CloudDebuggerOptions, DataflowWorkerLoggingOptions, DataflowProfilingOptions,
- PubsubOptions {
-
- @Description("Project id. Required when running a Dataflow in the cloud. "
- + "See https://cloud.google.com/storage/docs/projects for further details.")
- @Override
- @Validation.Required
- @Default.InstanceFactory(DefaultProjectFactory.class)
- String getProject();
- @Override
- void setProject(String value);
-
- /**
- * GCS path for staging local files, e.g. gs://bucket/object
- *
- * <p>Must be a valid Cloud Storage URL, beginning with the prefix "gs://"
- *
- * <p>At least one of {@link PipelineOptions#getTempLocation()} or {@link #getStagingLocation()}
- * must be set. If {@link #getStagingLocation()} is not set, then the Dataflow
- * pipeline defaults to using {@link PipelineOptions#getTempLocation()}.
- */
- @Description("GCS path for staging local files, e.g. \"gs://bucket/object\". "
- + "Must be a valid Cloud Storage URL, beginning with the prefix \"gs://\". "
- + "At least one of stagingLocation or tempLocation must be set. If stagingLocation is unset, "
- + "defaults to using tempLocation.")
- String getStagingLocation();
- void setStagingLocation(String value);
-
- /**
- * The Dataflow job name is used as an idempotence key within the Dataflow service.
- * If there is an existing job that is currently active, another active job with the same
- * name will not be able to be created. Defaults to using the ApplicationName-UserName-Date.
- */
- @Description("The Dataflow job name is used as an idempotence key within the Dataflow service. "
- + "For each running job in the same GCP project, jobs with the same name cannot be created "
- + "unless the new job is an explicit update of the previous one. Defaults to using "
- + "ApplicationName-UserName-Date. The job name must match the regular expression "
- + "'[a-z]([-a-z0-9]{0,38}[a-z0-9])?'. The runner will automatically truncate the name of the "
- + "job and convert to lower case.")
- @Default.InstanceFactory(JobNameFactory.class)
- String getJobName();
- void setJobName(String value);
-
- /**
- * Whether to update the currently running pipeline with the same name as this one.
- */
- @Description(
- "If set, replace the existing pipeline with the name specified by --jobName with "
- + "this pipeline, preserving state.")
- boolean isUpdate();
- void setUpdate(boolean value);
-
- /**
- * Returns a normalized job name constructed from {@link ApplicationNameOptions#getAppName()}, the
- * local system user name (if available), and the current time. The normalization makes sure that
- * the job name matches the required pattern of [a-z]([-a-z0-9]*[a-z0-9])? and length limit of 40
- * characters.
- *
- * <p>This job name factory is only able to generate one unique name per second per application
- * and user combination.
- */
- public static class JobNameFactory implements DefaultValueFactory<String> {
- private static final DateTimeFormatter FORMATTER =
- DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC);
-
- @Override
- public String create(PipelineOptions options) {
- String appName = options.as(ApplicationNameOptions.class).getAppName();
- String normalizedAppName = appName == null || appName.length() == 0 ? "dataflow"
- : appName.toLowerCase()
- .replaceAll("[^a-z0-9]", "0")
- .replaceAll("^[^a-z]", "a");
- String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), "");
- String normalizedUserName = userName.toLowerCase()
- .replaceAll("[^a-z0-9]", "0");
- String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis());
- return normalizedAppName + "-" + normalizedUserName + "-" + datePart;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java
deleted file mode 100644
index 6b44e7a..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowPipelineWorkerPoolOptions.java
+++ /dev/null
@@ -1,258 +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.options;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-
-import java.util.List;
-
-/**
- * Options that are used to configure the Dataflow pipeline worker pool.
- */
-@Description("Options that are used to configure the Dataflow pipeline worker pool.")
-public interface DataflowPipelineWorkerPoolOptions extends PipelineOptions {
- /**
- * Number of workers to use when executing the Dataflow job. Note that selection of an autoscaling
- * algorithm other then {@code NONE} will affect the size of the worker pool. If left unspecified,
- * the Dataflow service will determine the number of workers.
- */
- @Description("Number of workers to use when executing the Dataflow job. Note that "
- + "selection of an autoscaling algorithm other then \"NONE\" will affect the "
- + "size of the worker pool. If left unspecified, the Dataflow service will "
- + "determine the number of workers.")
- int getNumWorkers();
- void setNumWorkers(int value);
-
- /**
- * Type of autoscaling algorithm to use.
- */
- @Experimental(Experimental.Kind.AUTOSCALING)
- public enum AutoscalingAlgorithmType {
- /** Use numWorkers machines. Do not autoscale the worker pool. */
- NONE("AUTOSCALING_ALGORITHM_NONE"),
-
- @Deprecated
- BASIC("AUTOSCALING_ALGORITHM_BASIC"),
-
- /** Autoscale the workerpool based on throughput (up to maxNumWorkers). */
- THROUGHPUT_BASED("AUTOSCALING_ALGORITHM_BASIC");
-
- private final String algorithm;
-
- private AutoscalingAlgorithmType(String algorithm) {
- this.algorithm = algorithm;
- }
-
- /** Returns the string representation of this type. */
- public String getAlgorithm() {
- return this.algorithm;
- }
- }
-
- /**
- * [Experimental] The autoscaling algorithm to use for the workerpool.
- *
- * <ul>
- * <li>NONE: does not change the size of the worker pool.</li>
- * <li>BASIC: autoscale the worker pool size up to maxNumWorkers until the job completes.</li>
- * <li>THROUGHPUT_BASED: autoscale the workerpool based on throughput (up to maxNumWorkers).
- * </li>
- * </ul>
- */
- @Description("[Experimental] The autoscaling algorithm to use for the workerpool. "
- + "NONE: does not change the size of the worker pool. "
- + "BASIC (deprecated): autoscale the worker pool size up to maxNumWorkers until the job "
- + "completes. "
- + "THROUGHPUT_BASED: autoscale the workerpool based on throughput (up to maxNumWorkers).")
- @Experimental(Experimental.Kind.AUTOSCALING)
- AutoscalingAlgorithmType getAutoscalingAlgorithm();
- void setAutoscalingAlgorithm(AutoscalingAlgorithmType value);
-
- /**
- * The maximum number of workers to use for the workerpool. This options limits the size of the
- * workerpool for the lifetime of the job, including
- * <a href="https://cloud.google.com/dataflow/pipelines/updating-a-pipeline">pipeline updates</a>.
- * If left unspecified, the Dataflow service will compute a ceiling.
- */
- @Description("The maximum number of workers to use for the workerpool. This options limits the "
- + "size of the workerpool for the lifetime of the job, including pipeline updates. "
- + "If left unspecified, the Dataflow service will compute a ceiling.")
- int getMaxNumWorkers();
- void setMaxNumWorkers(int value);
-
- /**
- * Remote worker disk size, in gigabytes, or 0 to use the default size.
- */
- @Description("Remote worker disk size, in gigabytes, or 0 to use the default size.")
- int getDiskSizeGb();
- void setDiskSizeGb(int value);
-
- /**
- * Docker container image that executes Dataflow worker harness, residing in Google Container
- * Registry.
- */
- @Default.InstanceFactory(WorkerHarnessContainerImageFactory.class)
- @Description("Docker container image that executes Dataflow worker harness, residing in Google "
- + " Container Registry.")
- @Hidden
- String getWorkerHarnessContainerImage();
- void setWorkerHarnessContainerImage(String value);
-
- /**
- * Returns the default Docker container image that executes Dataflow worker harness, residing in
- * Google Container Registry.
- */
- public static class WorkerHarnessContainerImageFactory
- implements DefaultValueFactory<String> {
- @Override
- public String create(PipelineOptions options) {
- DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
- if (dataflowOptions.isStreaming()) {
- return DataflowPipelineRunner.STREAMING_WORKER_HARNESS_CONTAINER_IMAGE;
- } else {
- return DataflowPipelineRunner.BATCH_WORKER_HARNESS_CONTAINER_IMAGE;
- }
- }
- }
-
- /**
- * GCE <a href="https://cloud.google.com/compute/docs/networking">network</a> for launching
- * workers.
- *
- * <p>Default is up to the Dataflow service.
- */
- @Description("GCE network for launching workers. For more information, see the reference "
- + "documentation https://cloud.google.com/compute/docs/networking. "
- + "Default is up to the Dataflow service.")
- String getNetwork();
- void setNetwork(String value);
-
- /**
- * GCE <a href="https://cloud.google.com/compute/docs/networking">subnetwork</a> for launching
- * workers.
- *
- * <p>Default is up to the Dataflow service. Expected format is
- * regions/REGION/subnetworks/SUBNETWORK.
- *
- * <p>You may also need to specify network option.
- */
- @Description("GCE subnetwork for launching workers. For more information, see the reference "
- + "documentation https://cloud.google.com/compute/docs/networking. "
- + "Default is up to the Dataflow service.")
- String getSubnetwork();
- void setSubnetwork(String value);
-
- /**
- * GCE <a href="https://developers.google.com/compute/docs/zones"
- * >availability zone</a> for launching workers.
- *
- * <p>Default is up to the Dataflow service.
- */
- @Description("GCE availability zone for launching workers. See "
- + "https://developers.google.com/compute/docs/zones for a list of valid options. "
- + "Default is up to the Dataflow service.")
- String getZone();
- void setZone(String value);
-
- /**
- * Machine type to create Dataflow worker VMs as.
- *
- * <p>See <a href="https://cloud.google.com/compute/docs/machine-types">GCE machine types</a>
- * for a list of valid options.
- *
- * <p>If unset, the Dataflow service will choose a reasonable default.
- */
- @Description("Machine type to create Dataflow worker VMs as. See "
- + "https://cloud.google.com/compute/docs/machine-types for a list of valid options. "
- + "If unset, the Dataflow service will choose a reasonable default.")
- String getWorkerMachineType();
- void setWorkerMachineType(String value);
-
- /**
- * The policy for tearing down the workers spun up by the service.
- */
- public enum TeardownPolicy {
- /**
- * All VMs created for a Dataflow job are deleted when the job finishes, regardless of whether
- * it fails or succeeds.
- */
- TEARDOWN_ALWAYS("TEARDOWN_ALWAYS"),
- /**
- * All VMs created for a Dataflow job are left running when the job finishes, regardless of
- * whether it fails or succeeds.
- */
- TEARDOWN_NEVER("TEARDOWN_NEVER"),
- /**
- * All VMs created for a Dataflow job are deleted when the job succeeds, but are left running
- * when it fails. (This is typically used for debugging failing jobs by SSHing into the
- * workers.)
- */
- TEARDOWN_ON_SUCCESS("TEARDOWN_ON_SUCCESS");
-
- private final String teardownPolicy;
-
- private TeardownPolicy(String teardownPolicy) {
- this.teardownPolicy = teardownPolicy;
- }
-
- public String getTeardownPolicyName() {
- return this.teardownPolicy;
- }
- }
-
- /**
- * The teardown policy for the VMs.
- *
- * <p>If unset, the Dataflow service will choose a reasonable default.
- */
- @Description("The teardown policy for the VMs. If unset, the Dataflow service will "
- + "choose a reasonable default.")
- TeardownPolicy getTeardownPolicy();
- void setTeardownPolicy(TeardownPolicy value);
-
- /**
- * List of local files to make available to workers.
- *
- * <p>Files are placed on the worker's classpath.
- *
- * <p>The default value is the list of jars from the main program's classpath.
- */
- @Description("Files to stage on GCS and make available to workers. "
- + "Files are placed on the worker's classpath. "
- + "The default value is all files from the classpath.")
- @JsonIgnore
- List<String> getFilesToStage();
- void setFilesToStage(List<String> value);
-
- /**
- * Specifies what type of persistent disk should be used. The value should be a full or partial
- * URL of a disk type resource, e.g., zones/us-central1-f/disks/pd-standard. For
- * more information, see the
- * <a href="https://cloud.google.com/compute/docs/reference/latest/diskTypes">API reference
- * documentation for DiskTypes</a>.
- */
- @Description("Specifies what type of persistent disk should be used. The value should be a full "
- + "or partial URL of a disk type resource, e.g., zones/us-central1-f/disks/pd-standard. For "
- + "more information, see the API reference documentation for DiskTypes: "
- + "https://cloud.google.com/compute/docs/reference/latest/diskTypes")
- String getWorkerDiskType();
- void setWorkerDiskType(String value);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java
deleted file mode 100644
index da51230..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowProfilingOptions.java
+++ /dev/null
@@ -1,48 +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.options;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import java.util.HashMap;
-
-/**
- * Options for controlling profiling of pipeline execution.
- */
-@Description("[Experimental] Used to configure profiling of the Dataflow pipeline")
-@Experimental
-@Hidden
-public interface DataflowProfilingOptions {
-
- @Description("Whether to periodically dump profiling information to local disk.\n"
- + "WARNING: Enabling this option may fill local disk with profiling information.")
- boolean getEnableProfilingAgent();
- void setEnableProfilingAgent(boolean enabled);
-
- @Description(
- "[INTERNAL] Additional configuration for the profiling agent. Not typically necessary.")
- @Hidden
- DataflowProfilingAgentConfiguration getProfilingAgentConfiguration();
- void setProfilingAgentConfiguration(DataflowProfilingAgentConfiguration configuration);
-
- /**
- * Configuration the for profiling agent.
- */
- public static class DataflowProfilingAgentConfiguration extends HashMap<String, Object> {
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java
deleted file mode 100644
index a28b756..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerHarnessOptions.java
+++ /dev/null
@@ -1,51 +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.options;
-
-/**
- * Options that are used exclusively within the Dataflow worker harness.
- * These options have no effect at pipeline creation time.
- */
-@Description("[Internal] Options that are used exclusively within the Dataflow worker harness. "
- + "These options have no effect at pipeline creation time.")
-@Hidden
-public interface DataflowWorkerHarnessOptions extends DataflowPipelineOptions {
- /**
- * The identity of the worker running this pipeline.
- */
- @Description("The identity of the worker running this pipeline.")
- String getWorkerId();
- void setWorkerId(String value);
-
- /**
- * The identity of the Dataflow job.
- */
- @Description("The identity of the Dataflow job.")
- String getJobId();
- void setJobId(String value);
-
- /**
- * The size of the worker's in-memory cache, in megabytes.
- *
- * <p>Currently, this cache is used for storing read values of side inputs.
- */
- @Description("The size of the worker's in-memory cache, in megabytes.")
- @Default.Integer(100)
- Integer getWorkerCacheMb();
- void setWorkerCacheMb(Integer value);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java
deleted file mode 100644
index dc840d8..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptions.java
+++ /dev/null
@@ -1,155 +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.options;
-
-import com.google.common.base.Preconditions;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Options that are used to control logging configuration on the Dataflow worker.
- */
-@Description("Options that are used to control logging configuration on the Dataflow worker.")
-public interface DataflowWorkerLoggingOptions extends PipelineOptions {
- /**
- * The set of log levels that can be used on the Dataflow worker.
- */
- public enum Level {
- DEBUG, ERROR, INFO, TRACE, WARN
- }
-
- /**
- * This option controls the default log level of all loggers without a log level override.
- */
- @Description("Controls the default log level of all loggers without a log level override.")
- @Default.Enum("INFO")
- Level getDefaultWorkerLogLevel();
- void setDefaultWorkerLogLevel(Level level);
-
- /**
- * This option controls the log levels for specifically named loggers.
- *
- * <p>Later options with equivalent names override earlier options.
- *
- * <p>See {@link WorkerLogLevelOverrides} for more information on how to configure logging
- * on a per {@link Class}, {@link Package}, or name basis. If used from the command line,
- * the expected format is {"Name":"Level",...}, further details on
- * {@link WorkerLogLevelOverrides#from}.
- */
- @Description("This option controls the log levels for specifically named loggers. "
- + "The expected format is {\"Name\":\"Level\",...}. The Dataflow worker uses "
- + "java.util.logging, which supports a logging hierarchy based off of names that are '.' "
- + "separated. For example, by specifying the value {\"a.b.c.Foo\":\"DEBUG\"}, the logger "
- + "for the class 'a.b.c.Foo' will be configured to output logs at the DEBUG level. "
- + "Similarly, by specifying the value {\"a.b.c\":\"WARN\"}, all loggers underneath the "
- + "'a.b.c' package will be configured to output logs at the WARN level. Also, note that "
- + "when multiple overrides are specified, the exact name followed by the closest parent "
- + "takes precedence.")
- WorkerLogLevelOverrides getWorkerLogLevelOverrides();
- void setWorkerLogLevelOverrides(WorkerLogLevelOverrides value);
-
- /**
- * Defines a log level override for a specific class, package, or name.
- *
- * <p>{@code java.util.logging} is used on the Dataflow worker harness and supports
- * a logging hierarchy based off of names that are "." separated. It is a common
- * pattern to have the logger for a given class share the same name as the class itself.
- * Given the classes {@code a.b.c.Foo}, {@code a.b.c.Xyz}, and {@code a.b.Bar}, with
- * loggers named {@code "a.b.c.Foo"}, {@code "a.b.c.Xyz"}, and {@code "a.b.Bar"} respectively,
- * we can override the log levels:
- * <ul>
- * <li>for {@code Foo} by specifying the name {@code "a.b.c.Foo"} or the {@link Class}
- * representing {@code a.b.c.Foo}.
- * <li>for {@code Foo}, {@code Xyz}, and {@code Bar} by specifying the name {@code "a.b"} or
- * the {@link Package} representing {@code a.b}.
- * <li>for {@code Foo} and {@code Bar} by specifying both of their names or classes.
- * </ul>
- * Note that by specifying multiple overrides, the exact name followed by the closest parent
- * takes precedence.
- */
- public static class WorkerLogLevelOverrides extends HashMap<String, Level> {
- /**
- * Overrides the default log level for the passed in class.
- *
- * <p>This is equivalent to calling
- * {@link #addOverrideForName(String, DataflowWorkerLoggingOptions.Level)}
- * and passing in the {@link Class#getName() class name}.
- */
- public WorkerLogLevelOverrides addOverrideForClass(Class<?> klass, Level level) {
- Preconditions.checkNotNull(klass, "Expected class to be not null.");
- addOverrideForName(klass.getName(), level);
- return this;
- }
-
- /**
- * Overrides the default log level for the passed in package.
- *
- * <p>This is equivalent to calling
- * {@link #addOverrideForName(String, DataflowWorkerLoggingOptions.Level)}
- * and passing in the {@link Package#getName() package name}.
- */
- public WorkerLogLevelOverrides addOverrideForPackage(Package pkg, Level level) {
- Preconditions.checkNotNull(pkg, "Expected package to be not null.");
- addOverrideForName(pkg.getName(), level);
- return this;
- }
-
- /**
- * Overrides the default log level for the passed in name.
- *
- * <p>Note that because of the hierarchical nature of logger names, this will
- * override the log level of all loggers that have the passed in name or
- * a parent logger that has the passed in name.
- */
- public WorkerLogLevelOverrides addOverrideForName(String name, Level level) {
- Preconditions.checkNotNull(name, "Expected name to be not null.");
- Preconditions.checkNotNull(level,
- "Expected level to be one of %s.", Arrays.toString(Level.values()));
- put(name, level);
- return this;
- }
-
- /**
- * Expects a map keyed by logger {@code Name}s with values representing {@code Level}s.
- * The {@code Name} generally represents the fully qualified Java
- * {@link Class#getName() class name}, or fully qualified Java
- * {@link Package#getName() package name}, or custom logger name. The {@code Level}
- * represents the log level and must be one of {@link Level}.
- */
- @JsonCreator
- public static WorkerLogLevelOverrides from(Map<String, String> values) {
- Preconditions.checkNotNull(values, "Expected values to be not null.");
- WorkerLogLevelOverrides overrides = new WorkerLogLevelOverrides();
- for (Map.Entry<String, String> entry : values.entrySet()) {
- try {
- overrides.addOverrideForName(entry.getKey(), Level.valueOf(entry.getValue()));
- } catch (IllegalArgumentException e) {
- throw new IllegalArgumentException(String.format(
- "Unsupported log level '%s' requested for %s. Must be one of %s.",
- entry.getValue(), entry.getKey(), Arrays.toString(Level.values())));
- }
-
- }
- return overrides;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java
deleted file mode 100644
index 0f4126f..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunner.java
+++ /dev/null
@@ -1,185 +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.runners;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.options.BlockingDataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.MonitoringUtil;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link PipelineRunner} that's like {@link DataflowPipelineRunner}
- * but that waits for the launched job to finish.
- *
- * <p>Prints out job status updates and console messages while it waits.
- *
- * <p>Returns the final job state, or throws an exception if the job
- * fails or cannot be monitored.
- *
- * <p><h3>Permissions</h3>
- * When reading from a Dataflow source or writing to a Dataflow sink using
- * {@code BlockingDataflowPipelineRunner}, the Google cloud services account and the Google compute
- * engine service account of the GCP project running the Dataflow Job will need access to the
- * corresponding source/sink.
- *
- * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
- * Dataflow Security and Permissions</a> for more details.
- */
-public class BlockingDataflowPipelineRunner extends
- PipelineRunner<DataflowPipelineJob> {
- private static final Logger LOG = LoggerFactory.getLogger(BlockingDataflowPipelineRunner.class);
-
- // Defaults to an infinite wait period.
- // TODO: make this configurable after removal of option map.
- private static final long BUILTIN_JOB_TIMEOUT_SEC = -1L;
-
- private final DataflowPipelineRunner dataflowPipelineRunner;
- private final BlockingDataflowPipelineOptions options;
-
- protected BlockingDataflowPipelineRunner(
- DataflowPipelineRunner internalRunner,
- BlockingDataflowPipelineOptions options) {
- this.dataflowPipelineRunner = internalRunner;
- this.options = options;
- }
-
- /**
- * Constructs a runner from the provided options.
- */
- public static BlockingDataflowPipelineRunner fromOptions(
- PipelineOptions options) {
- BlockingDataflowPipelineOptions dataflowOptions =
- PipelineOptionsValidator.validate(BlockingDataflowPipelineOptions.class, options);
- DataflowPipelineRunner dataflowPipelineRunner =
- DataflowPipelineRunner.fromOptions(dataflowOptions);
-
- return new BlockingDataflowPipelineRunner(dataflowPipelineRunner, dataflowOptions);
- }
-
- /**
- * {@inheritDoc}
- *
- * @throws DataflowJobExecutionException if there is an exception during job execution.
- * @throws DataflowServiceException if there is an exception retrieving information about the job.
- */
- @Override
- public DataflowPipelineJob run(Pipeline p) {
- final DataflowPipelineJob job = dataflowPipelineRunner.run(p);
-
- // We ignore the potential race condition here (Ctrl-C after job submission but before the
- // shutdown hook is registered). Even if we tried to do something smarter (eg., SettableFuture)
- // the run method (which produces the job) could fail or be Ctrl-C'd before it had returned a
- // job. The display of the command to cancel the job is best-effort anyways -- RPC's could fail,
- // etc. If the user wants to verify the job was cancelled they should look at the job status.
- Thread shutdownHook = new Thread() {
- @Override
- public void run() {
- LOG.warn("Job is already running in Google Cloud Platform, Ctrl-C will not cancel it.\n"
- + "To cancel the job in the cloud, run:\n> {}",
- MonitoringUtil.getGcloudCancelCommand(options, job.getJobId()));
- }
- };
-
- try {
- Runtime.getRuntime().addShutdownHook(shutdownHook);
-
- @Nullable
- State result;
- try {
- result = job.waitToFinish(
- BUILTIN_JOB_TIMEOUT_SEC, TimeUnit.SECONDS,
- new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
- } catch (IOException | InterruptedException ex) {
- if (ex instanceof InterruptedException) {
- Thread.currentThread().interrupt();
- }
- LOG.debug("Exception caught while retrieving status for job {}", job.getJobId(), ex);
- throw new DataflowServiceException(
- job, "Exception caught while retrieving status for job " + job.getJobId(), ex);
- }
-
- if (result == null) {
- throw new DataflowServiceException(
- job, "Timed out while retrieving status for job " + job.getJobId());
- }
-
- LOG.info("Job finished with status {}", result);
- if (!result.isTerminal()) {
- throw new IllegalStateException("Expected terminal state for job " + job.getJobId()
- + ", got " + result);
- }
-
- if (result == State.DONE) {
- return job;
- } else if (result == State.UPDATED) {
- DataflowPipelineJob newJob = job.getReplacedByJob();
- LOG.info("Job {} has been updated and is running as the new job with id {}."
- + "To access the updated job on the Dataflow monitoring console, please navigate to {}",
- job.getJobId(),
- newJob.getJobId(),
- MonitoringUtil.getJobMonitoringPageURL(newJob.getProjectId(), newJob.getJobId()));
- throw new DataflowJobUpdatedException(
- job,
- String.format("Job %s updated; new job is %s.", job.getJobId(), newJob.getJobId()),
- newJob);
- } else if (result == State.CANCELLED) {
- String message = String.format("Job %s cancelled by user", job.getJobId());
- LOG.info(message);
- throw new DataflowJobCancelledException(job, message);
- } else {
- throw new DataflowJobExecutionException(job, "Job " + job.getJobId()
- + " failed with status " + result);
- }
- } finally {
- Runtime.getRuntime().removeShutdownHook(shutdownHook);
- }
- }
-
- @Override
- public <OutputT extends POutput, InputT extends PInput> OutputT apply(
- PTransform<InputT, OutputT> transform, InputT input) {
- return dataflowPipelineRunner.apply(transform, input);
- }
-
- /**
- * Sets callbacks to invoke during execution. See {@link DataflowPipelineRunnerHooks}.
- */
- @Experimental
- public void setHooks(DataflowPipelineRunnerHooks hooks) {
- this.dataflowPipelineRunner.setHooks(hooks);
- }
-
- @Override
- public String toString() {
- return "BlockingDataflowPipelineRunner#" + options.getJobName();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java
deleted file mode 100644
index 728baa7..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyExistsException.java
+++ /dev/null
@@ -1,35 +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.runners;
-
-/**
- * An exception that is thrown if the unique job name constraint of the Dataflow
- * service is broken because an existing job with the same job name is currently active.
- * The {@link DataflowPipelineJob} contained within this exception contains information
- * about the pre-existing job.
- */
-public class DataflowJobAlreadyExistsException extends DataflowJobException {
- /**
- * Create a new {@code DataflowJobAlreadyExistsException} with the specified {@link
- * DataflowPipelineJob} and message.
- */
- public DataflowJobAlreadyExistsException(
- DataflowPipelineJob job, String message) {
- super(job, message, null);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java
deleted file mode 100644
index 4d2025f..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobAlreadyUpdatedException.java
+++ /dev/null
@@ -1,34 +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.runners;
-
-/**
- * An exception that is thrown if the existing job has already been updated within the Dataflow
- * service and is no longer able to be updated. The {@link DataflowPipelineJob} contained within
- * this exception contains information about the pre-existing updated job.
- */
-public class DataflowJobAlreadyUpdatedException extends DataflowJobException {
- /**
- * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
- * DataflowPipelineJob} and message.
- */
- public DataflowJobAlreadyUpdatedException(
- DataflowPipelineJob job, String message) {
- super(job, message, null);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java
deleted file mode 100644
index cc002f1..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobCancelledException.java
+++ /dev/null
@@ -1,39 +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.runners;
-
-/**
- * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.
- */
-public class DataflowJobCancelledException extends DataflowJobException {
- /**
- * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
- * DataflowPipelineJob} and message.
- */
- public DataflowJobCancelledException(DataflowPipelineJob job, String message) {
- super(job, message, null);
- }
-
- /**
- * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
- * DataflowPipelineJob}, message, and cause.
- */
- public DataflowJobCancelledException(DataflowPipelineJob job, String message, Throwable cause) {
- super(job, message, cause);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java
deleted file mode 100644
index ed66fb7..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobException.java
+++ /dev/null
@@ -1,41 +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.runners;
-
-import java.util.Objects;
-
-import javax.annotation.Nullable;
-
-/**
- * A {@link RuntimeException} that contains information about a {@link DataflowPipelineJob}.
- */
-public abstract class DataflowJobException extends RuntimeException {
- private final DataflowPipelineJob job;
-
- DataflowJobException(DataflowPipelineJob job, String message, @Nullable Throwable cause) {
- super(message, cause);
- this.job = Objects.requireNonNull(job);
- }
-
- /**
- * Returns the failed job.
- */
- public DataflowPipelineJob getJob() {
- return job;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java
deleted file mode 100644
index bc8a903..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobExecutionException.java
+++ /dev/null
@@ -1,35 +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.runners;
-
-import javax.annotation.Nullable;
-
-/**
- * Signals that a job run by a {@link BlockingDataflowPipelineRunner} fails during execution, and
- * provides access to the failed job.
- */
-public class DataflowJobExecutionException extends DataflowJobException {
- DataflowJobExecutionException(DataflowPipelineJob job, String message) {
- this(job, message, null);
- }
-
- DataflowJobExecutionException(
- DataflowPipelineJob job, String message, @Nullable Throwable cause) {
- super(job, message, cause);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java
deleted file mode 100644
index 65c327f..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowJobUpdatedException.java
+++ /dev/null
@@ -1,51 +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.runners;
-
-/**
- * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.
- */
-public class DataflowJobUpdatedException extends DataflowJobException {
- private DataflowPipelineJob replacedByJob;
-
- /**
- * Create a new {@code DataflowJobUpdatedException} with the specified original {@link
- * DataflowPipelineJob}, message, and replacement {@link DataflowPipelineJob}.
- */
- public DataflowJobUpdatedException(
- DataflowPipelineJob job, String message, DataflowPipelineJob replacedByJob) {
- this(job, message, replacedByJob, null);
- }
-
- /**
- * Create a new {@code DataflowJobUpdatedException} with the specified original {@link
- * DataflowPipelineJob}, message, replacement {@link DataflowPipelineJob}, and cause.
- */
- public DataflowJobUpdatedException(
- DataflowPipelineJob job, String message, DataflowPipelineJob replacedByJob, Throwable cause) {
- super(job, message, cause);
- this.replacedByJob = replacedByJob;
- }
-
- /**
- * The new job that replaces the job terminated with this exception.
- */
- public DataflowPipelineJob getReplacedByJob() {
- return replacedByJob;
- }
-}
[08/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java
new file mode 100644
index 0000000..cf9a95a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.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.dataflow;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.ServiceLoader;
+
+/** Tests for {@link DataflowPipelineRegistrar}. */
+@RunWith(JUnit4.class)
+public class DataflowPipelineRegistrarTest {
+ @Test
+ public void testCorrectOptionsAreReturned() {
+ assertEquals(ImmutableList.of(DataflowPipelineOptions.class,
+ BlockingDataflowPipelineOptions.class),
+ new DataflowPipelineRegistrar.Options().getPipelineOptions());
+ }
+
+ @Test
+ public void testCorrectRunnersAreReturned() {
+ assertEquals(ImmutableList.of(DataflowPipelineRunner.class,
+ BlockingDataflowPipelineRunner.class),
+ new DataflowPipelineRegistrar.Runner().getPipelineRunners());
+ }
+
+ @Test
+ public void testServiceLoaderForOptions() {
+ for (PipelineOptionsRegistrar registrar :
+ Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
+ if (registrar instanceof DataflowPipelineRegistrar.Options) {
+ return;
+ }
+ }
+ fail("Expected to find " + DataflowPipelineRegistrar.Options.class);
+ }
+
+ @Test
+ public void testServiceLoaderForRunner() {
+ for (PipelineRunnerRegistrar registrar :
+ Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
+ if (registrar instanceof DataflowPipelineRegistrar.Runner) {
+ return;
+ }
+ }
+ fail("Expected to find " + DataflowPipelineRegistrar.Runner.class);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java
new file mode 100644
index 0000000..79e281e
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerTest.java
@@ -0,0 +1,1401 @@
+/*
+ * 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.dataflow;
+
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.startsWith;
+import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner.BatchViewAsList;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner.BatchViewAsMap;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner.BatchViewAsMultimap;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner.TransformedMap;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.AvroSource;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.dataflow.TestCountingSource;
+import org.apache.beam.sdk.runners.worker.IsmFormat;
+import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecord;
+import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecordCoder;
+import org.apache.beam.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.PTransform;
+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.util.CoderUtils;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.ReleaseInfo;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.UserCodeException;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.ListJobsResponse;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import org.hamcrest.Description;
+import org.hamcrest.Matchers;
+import org.hamcrest.TypeSafeMatcher;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.internal.matchers.ThrowableMessageMatcher;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * Tests for the {@link DataflowPipelineRunner}.
+ */
+@RunWith(JUnit4.class)
+public class DataflowPipelineRunnerTest {
+
+ private static final String PROJECT_ID = "some-project";
+
+ @Rule
+ public TemporaryFolder tmpFolder = new TemporaryFolder();
+ @Rule
+ public ExpectedException thrown = ExpectedException.none();
+
+ // Asserts that the given Job has all expected fields set.
+ private static void assertValidJob(Job job) {
+ assertNull(job.getId());
+ assertNull(job.getCurrentState());
+ assertTrue(Pattern.matches("[a-z]([-a-z0-9]*[a-z0-9])?", job.getName()));
+ }
+
+ private Pipeline buildDataflowPipeline(DataflowPipelineOptions options) {
+ options.setStableUniqueNames(CheckEnabled.ERROR);
+ options.setRunner(DataflowPipelineRunner.class);
+ Pipeline p = Pipeline.create(options);
+
+ p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
+ .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
+
+ return p;
+ }
+
+ private static Dataflow buildMockDataflow(
+ final ArgumentCaptor<Job> jobCaptor) throws IOException {
+ Dataflow mockDataflowClient = mock(Dataflow.class);
+ Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
+ Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
+ Dataflow.Projects.Jobs.Create mockRequest =
+ mock(Dataflow.Projects.Jobs.Create.class);
+ Dataflow.Projects.Jobs.List mockList = mock(Dataflow.Projects.Jobs.List.class);
+
+ when(mockDataflowClient.projects()).thenReturn(mockProjects);
+ when(mockProjects.jobs()).thenReturn(mockJobs);
+ when(mockJobs.create(eq(PROJECT_ID), jobCaptor.capture()))
+ .thenReturn(mockRequest);
+ when(mockJobs.list(eq(PROJECT_ID))).thenReturn(mockList);
+ when(mockList.setPageToken(anyString())).thenReturn(mockList);
+ when(mockList.execute())
+ .thenReturn(
+ new ListJobsResponse()
+ .setJobs(
+ Arrays.asList(
+ new Job()
+ .setName("oldjobname")
+ .setId("oldJobId")
+ .setCurrentState("JOB_STATE_RUNNING"))));
+
+ Job resultJob = new Job();
+ resultJob.setId("newid");
+ when(mockRequest.execute()).thenReturn(resultJob);
+ return mockDataflowClient;
+ }
+
+ private GcsUtil buildMockGcsUtil(boolean bucketExists) throws IOException {
+ GcsUtil mockGcsUtil = mock(GcsUtil.class);
+ when(mockGcsUtil.create(any(GcsPath.class), anyString()))
+ .then(new Answer<SeekableByteChannel>() {
+ @Override
+ public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
+ return FileChannel.open(
+ Files.createTempFile("channel-", ".tmp"),
+ StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
+ }
+ });
+
+ when(mockGcsUtil.isGcsPatternSupported(anyString())).thenReturn(true);
+ when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer<List<GcsPath>>() {
+ @Override
+ public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
+ return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
+ }
+ });
+ when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(bucketExists);
+ return mockGcsUtil;
+ }
+
+ private DataflowPipelineOptions buildPipelineOptions() throws IOException {
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+ return buildPipelineOptions(jobCaptor);
+ }
+
+ private DataflowPipelineOptions buildPipelineOptions(
+ ArgumentCaptor<Job> jobCaptor) throws IOException {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setProject(PROJECT_ID);
+ options.setTempLocation("gs://somebucket/some/path");
+ // Set FILES_PROPERTY to empty to prevent a default value calculated from classpath.
+ options.setFilesToStage(new LinkedList<String>());
+ options.setDataflowClient(buildMockDataflow(jobCaptor));
+ options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+ options.setGcpCredential(new TestCredential());
+ return options;
+ }
+
+ @Test
+ public void testFromOptionsWithUppercaseConvertsToLowercase() throws Exception {
+ String mixedCase = "ThisJobNameHasMixedCase";
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+ DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+ options.setJobName(mixedCase);
+
+ DataflowPipelineRunner runner = DataflowPipelineRunner.fromOptions(options);
+ assertThat(options.getJobName(), equalTo(mixedCase.toLowerCase()));
+ }
+
+ @Test
+ public void testRun() throws IOException {
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+ DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+ Pipeline p = buildDataflowPipeline(options);
+ DataflowPipelineJob job = (DataflowPipelineJob) p.run();
+ assertEquals("newid", job.getJobId());
+ assertValidJob(jobCaptor.getValue());
+ }
+
+ @Test
+ public void testRunReturnDifferentRequestId() throws IOException {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ Dataflow mockDataflowClient = options.getDataflowClient();
+ Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
+ when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
+ .thenReturn(mockRequest);
+ Job resultJob = new Job();
+ resultJob.setId("newid");
+ // Return a different request id.
+ resultJob.setClientRequestId("different_request_id");
+ when(mockRequest.execute()).thenReturn(resultJob);
+
+ Pipeline p = buildDataflowPipeline(options);
+ try {
+ p.run();
+ fail("Expected DataflowJobAlreadyExistsException");
+ } catch (DataflowJobAlreadyExistsException expected) {
+ assertThat(expected.getMessage(),
+ containsString("If you want to submit a second job, try again by setting a "
+ + "different name using --jobName."));
+ assertEquals(expected.getJob().getJobId(), resultJob.getId());
+ }
+ }
+
+ @Test
+ public void testUpdate() throws IOException {
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+ DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+ options.setUpdate(true);
+ options.setJobName("oldJobName");
+ Pipeline p = buildDataflowPipeline(options);
+ DataflowPipelineJob job = (DataflowPipelineJob) p.run();
+ assertEquals("newid", job.getJobId());
+ assertValidJob(jobCaptor.getValue());
+ }
+
+ @Test
+ public void testUpdateNonExistentPipeline() throws IOException {
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("Could not find running job named badjobname");
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setUpdate(true);
+ options.setJobName("badJobName");
+ Pipeline p = buildDataflowPipeline(options);
+ p.run();
+ }
+
+ @Test
+ public void testUpdateAlreadyUpdatedPipeline() throws IOException {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setUpdate(true);
+ options.setJobName("oldJobName");
+ Dataflow mockDataflowClient = options.getDataflowClient();
+ Dataflow.Projects.Jobs.Create mockRequest = mock(Dataflow.Projects.Jobs.Create.class);
+ when(mockDataflowClient.projects().jobs().create(eq(PROJECT_ID), any(Job.class)))
+ .thenReturn(mockRequest);
+ final Job resultJob = new Job();
+ resultJob.setId("newid");
+ // Return a different request id.
+ resultJob.setClientRequestId("different_request_id");
+ when(mockRequest.execute()).thenReturn(resultJob);
+
+ Pipeline p = buildDataflowPipeline(options);
+
+ thrown.expect(DataflowJobAlreadyUpdatedException.class);
+ thrown.expect(new TypeSafeMatcher<DataflowJobAlreadyUpdatedException>() {
+ @Override
+ public void describeTo(Description description) {
+ description.appendText("Expected job ID: " + resultJob.getId());
+ }
+
+ @Override
+ protected boolean matchesSafely(DataflowJobAlreadyUpdatedException item) {
+ return resultJob.getId().equals(item.getJob().getJobId());
+ }
+ });
+ thrown.expectMessage("The job named oldjobname with id: oldJobId has already been updated "
+ + "into job id: newid and cannot be updated again.");
+ p.run();
+ }
+
+ @Test
+ public void testRunWithFiles() throws IOException {
+ // Test that the function DataflowPipelineRunner.stageFiles works as
+ // expected.
+ GcsUtil mockGcsUtil = buildMockGcsUtil(true /* bucket exists */);
+ final String gcsStaging = "gs://somebucket/some/path";
+ final String gcsTemp = "gs://somebucket/some/temp/path";
+ final String cloudDataflowDataset = "somedataset";
+
+ // Create some temporary files.
+ File temp1 = File.createTempFile("DataflowPipelineRunnerTest", "txt");
+ temp1.deleteOnExit();
+ File temp2 = File.createTempFile("DataflowPipelineRunnerTest2", "txt");
+ temp2.deleteOnExit();
+
+ String overridePackageName = "alias.txt";
+
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setFilesToStage(ImmutableList.of(
+ temp1.getAbsolutePath(),
+ overridePackageName + "=" + temp2.getAbsolutePath()));
+ options.setStagingLocation(gcsStaging);
+ options.setTempLocation(gcsTemp);
+ options.setTempDatasetId(cloudDataflowDataset);
+ options.setProject(PROJECT_ID);
+ options.setJobName("job");
+ options.setDataflowClient(buildMockDataflow(jobCaptor));
+ options.setGcsUtil(mockGcsUtil);
+ options.setGcpCredential(new TestCredential());
+
+ Pipeline p = buildDataflowPipeline(options);
+
+ DataflowPipelineJob job = (DataflowPipelineJob) p.run();
+ assertEquals("newid", job.getJobId());
+
+ Job workflowJob = jobCaptor.getValue();
+ assertValidJob(workflowJob);
+
+ assertEquals(
+ 2,
+ workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().size());
+ DataflowPackage workflowPackage1 =
+ workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(0);
+ assertThat(workflowPackage1.getName(), startsWith(temp1.getName()));
+ DataflowPackage workflowPackage2 =
+ workflowJob.getEnvironment().getWorkerPools().get(0).getPackages().get(1);
+ assertEquals(overridePackageName, workflowPackage2.getName());
+
+ assertEquals(
+ "storage.googleapis.com/somebucket/some/temp/path",
+ workflowJob.getEnvironment().getTempStoragePrefix());
+ assertEquals(
+ cloudDataflowDataset,
+ workflowJob.getEnvironment().getDataset());
+ assertEquals(
+ ReleaseInfo.getReleaseInfo().getName(),
+ workflowJob.getEnvironment().getUserAgent().get("name"));
+ assertEquals(
+ ReleaseInfo.getReleaseInfo().getVersion(),
+ workflowJob.getEnvironment().getUserAgent().get("version"));
+ }
+
+ @Test
+ public void runWithDefaultFilesToStage() throws Exception {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setFilesToStage(null);
+ DataflowPipelineRunner.fromOptions(options);
+ assertTrue(!options.getFilesToStage().isEmpty());
+ }
+
+ @Test
+ public void detectClassPathResourceWithFileResources() throws Exception {
+ File file = tmpFolder.newFile("file");
+ File file2 = tmpFolder.newFile("file2");
+ URLClassLoader classLoader = new URLClassLoader(new URL[]{
+ file.toURI().toURL(),
+ file2.toURI().toURL()
+ });
+
+ assertEquals(ImmutableList.of(file.getAbsolutePath(), file2.getAbsolutePath()),
+ DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader));
+ }
+
+ @Test
+ public void detectClassPathResourcesWithUnsupportedClassLoader() {
+ ClassLoader mockClassLoader = Mockito.mock(ClassLoader.class);
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("Unable to use ClassLoader to detect classpath elements.");
+
+ DataflowPipelineRunner.detectClassPathResourcesToStage(mockClassLoader);
+ }
+
+ @Test
+ public void detectClassPathResourceWithNonFileResources() throws Exception {
+ String url = "http://www.google.com/all-the-secrets.jar";
+ URLClassLoader classLoader = new URLClassLoader(new URL[]{
+ new URL(url)
+ });
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("Unable to convert url (" + url + ") to file.");
+
+ DataflowPipelineRunner.detectClassPathResourcesToStage(classLoader);
+ }
+
+ @Test
+ public void testGcsStagingLocationInitialization() throws Exception {
+ // Test that the staging location is initialized correctly.
+ String gcsTemp = "gs://somebucket/some/temp/path";
+
+ // Set temp location (required), and check that staging location is set.
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setTempLocation(gcsTemp);
+ options.setProject(PROJECT_ID);
+ options.setGcpCredential(new TestCredential());
+ options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+ DataflowPipelineRunner.fromOptions(options);
+
+ assertNotNull(options.getStagingLocation());
+ }
+
+ @Test
+ public void testNonGcsFilePathInReadFailure() throws IOException {
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+ Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
+ p.apply(TextIO.Read.named("ReadMyNonGcsFile").from(tmpFolder.newFile().getPath()));
+
+ thrown.expectCause(Matchers.allOf(
+ instanceOf(IllegalArgumentException.class),
+ ThrowableMessageMatcher.hasMessage(
+ containsString("expected a valid 'gs://' path but was given"))));
+ p.run();
+ assertValidJob(jobCaptor.getValue());
+ }
+
+ @Test
+ public void testNonGcsFilePathInWriteFailure() throws IOException {
+ Pipeline p = buildDataflowPipeline(buildPipelineOptions());
+ PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
+ pc.apply(TextIO.Write.named("WriteMyNonGcsFile").to("/tmp/file"));
+ }
+
+ @Test
+ public void testMultiSlashGcsFileReadPath() throws IOException {
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+ Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
+ p.apply(TextIO.Read.named("ReadInvalidGcsFile")
+ .from("gs://bucket/tmp//file"));
+
+ thrown.expectCause(Matchers.allOf(
+ instanceOf(IllegalArgumentException.class),
+ ThrowableMessageMatcher.hasMessage(containsString("consecutive slashes"))));
+ p.run();
+ assertValidJob(jobCaptor.getValue());
+ }
+
+ @Test
+ public void testMultiSlashGcsFileWritePath() throws IOException {
+ Pipeline p = buildDataflowPipeline(buildPipelineOptions());
+ PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("consecutive slashes");
+ pc.apply(TextIO.Write.named("WriteInvalidGcsFile").to("gs://bucket/tmp//file"));
+ }
+
+ @Test
+ public void testInvalidTempLocation() throws IOException {
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+ DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+ options.setTempLocation("file://temp/location");
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
+ DataflowPipelineRunner.fromOptions(options);
+ assertValidJob(jobCaptor.getValue());
+ }
+
+ @Test
+ public void testInvalidStagingLocation() throws IOException {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setStagingLocation("file://my/staging/location");
+ try {
+ DataflowPipelineRunner.fromOptions(options);
+ fail("fromOptions should have failed");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
+ }
+ options.setStagingLocation("my/staging/location");
+ try {
+ DataflowPipelineRunner.fromOptions(options);
+ fail("fromOptions should have failed");
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(), containsString("expected a valid 'gs://' path but was given"));
+ }
+ }
+
+ @Test
+ public void testNonExistentTempLocation() throws IOException {
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+ GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
+ DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+ options.setGcsUtil(mockGcsUtil);
+ options.setTempLocation("gs://non-existent-bucket/location");
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage(containsString(
+ "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
+ DataflowPipelineRunner.fromOptions(options);
+ assertValidJob(jobCaptor.getValue());
+ }
+
+ @Test
+ public void testNonExistentStagingLocation() throws IOException {
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+ GcsUtil mockGcsUtil = buildMockGcsUtil(false /* bucket exists */);
+ DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+ options.setGcsUtil(mockGcsUtil);
+ options.setStagingLocation("gs://non-existent-bucket/location");
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage(containsString(
+ "Output path does not exist or is not writeable: gs://non-existent-bucket/location"));
+ DataflowPipelineRunner.fromOptions(options);
+ assertValidJob(jobCaptor.getValue());
+ }
+
+ @Test
+ public void testNoProjectFails() {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+
+ options.setRunner(DataflowPipelineRunner.class);
+ // Explicitly set to null to prevent the default instance factory from reading credentials
+ // from a user's environment, causing this test to fail.
+ options.setProject(null);
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("Project id");
+ thrown.expectMessage("when running a Dataflow in the cloud");
+
+ DataflowPipelineRunner.fromOptions(options);
+ }
+
+ @Test
+ public void testProjectId() throws IOException {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setProject("foo-12345");
+
+ options.setStagingLocation("gs://spam/ham/eggs");
+ options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+ options.setGcpCredential(new TestCredential());
+
+ DataflowPipelineRunner.fromOptions(options);
+ }
+
+ @Test
+ public void testProjectPrefix() throws IOException {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setProject("google.com:some-project-12345");
+
+ options.setStagingLocation("gs://spam/ham/eggs");
+ options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+ options.setGcpCredential(new TestCredential());
+
+ DataflowPipelineRunner.fromOptions(options);
+ }
+
+ @Test
+ public void testProjectNumber() throws IOException {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setProject("12345");
+
+ options.setStagingLocation("gs://spam/ham/eggs");
+ options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("Project ID");
+ thrown.expectMessage("project number");
+
+ DataflowPipelineRunner.fromOptions(options);
+ }
+
+ @Test
+ public void testProjectDescription() throws IOException {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setProject("some project");
+
+ options.setStagingLocation("gs://spam/ham/eggs");
+ options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("Project ID");
+ thrown.expectMessage("project description");
+
+ DataflowPipelineRunner.fromOptions(options);
+ }
+
+ @Test
+ public void testInvalidNumberOfWorkerHarnessThreads() throws IOException {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setProject("foo-12345");
+
+ options.setStagingLocation("gs://spam/ham/eggs");
+ options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+ options.as(DataflowPipelineDebugOptions.class).setNumberOfWorkerHarnessThreads(-1);
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("Number of worker harness threads");
+ thrown.expectMessage("Please make sure the value is non-negative.");
+
+ DataflowPipelineRunner.fromOptions(options);
+ }
+
+ @Test
+ public void testNoStagingLocationAndNoTempLocationFails() {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setProject("foo-project");
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage(
+ "Missing required value: at least one of tempLocation or stagingLocation must be set.");
+
+ DataflowPipelineRunner.fromOptions(options);
+ }
+
+ @Test
+ public void testStagingLocationAndNoTempLocationSucceeds() throws Exception {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setGcpCredential(new TestCredential());
+ options.setProject("foo-project");
+ options.setStagingLocation("gs://spam/ham/eggs");
+ options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+ DataflowPipelineRunner.fromOptions(options);
+ }
+
+ @Test
+ public void testTempLocationAndNoStagingLocationSucceeds() throws Exception {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setGcpCredential(new TestCredential());
+ options.setProject("foo-project");
+ options.setTempLocation("gs://spam/ham/eggs");
+ options.setGcsUtil(buildMockGcsUtil(true /* bucket exists */));
+
+ DataflowPipelineRunner.fromOptions(options);
+ }
+
+ @Test
+ public void testInvalidJobName() throws IOException {
+ List<String> invalidNames = Arrays.asList(
+ "invalid_name",
+ "0invalid",
+ "invalid-");
+ List<String> expectedReason = Arrays.asList(
+ "JobName invalid",
+ "JobName invalid",
+ "JobName invalid");
+
+ for (int i = 0; i < invalidNames.size(); ++i) {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setJobName(invalidNames.get(i));
+
+ try {
+ DataflowPipelineRunner.fromOptions(options);
+ fail("Expected IllegalArgumentException for jobName "
+ + options.getJobName());
+ } catch (IllegalArgumentException e) {
+ assertThat(e.getMessage(),
+ containsString(expectedReason.get(i)));
+ }
+ }
+ }
+
+ @Test
+ public void testValidJobName() throws IOException {
+ List<String> names = Arrays.asList("ok", "Ok", "A-Ok", "ok-123",
+ "this-one-is-fairly-long-01234567890123456789");
+
+ for (String name : names) {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setJobName(name);
+
+ DataflowPipelineRunner runner = DataflowPipelineRunner
+ .fromOptions(options);
+ assertNotNull(runner);
+ }
+ }
+
+ /**
+ * A fake PTransform for testing.
+ */
+ public static class TestTransform
+ extends PTransform<PCollection<Integer>, PCollection<Integer>> {
+ public boolean translated = false;
+
+ @Override
+ public PCollection<Integer> apply(PCollection<Integer> input) {
+ return PCollection.<Integer>createPrimitiveOutputInternal(
+ input.getPipeline(),
+ WindowingStrategy.globalDefault(),
+ input.isBounded());
+ }
+
+ @Override
+ protected Coder<?> getDefaultOutputCoder(PCollection<Integer> input) {
+ return input.getCoder();
+ }
+ }
+
+ @Test
+ public void testTransformTranslatorMissing() throws IOException {
+ // Test that we throw if we don't provide a translation.
+ ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
+
+ DataflowPipelineOptions options = buildPipelineOptions(jobCaptor);
+ Pipeline p = Pipeline.create(options);
+
+ p.apply(Create.of(Arrays.asList(1, 2, 3)))
+ .apply(new TestTransform());
+
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage(Matchers.containsString("no translator registered"));
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
+ assertValidJob(jobCaptor.getValue());
+ }
+
+ @Test
+ public void testTransformTranslator() throws IOException {
+ // Test that we can provide a custom translation
+ DataflowPipelineOptions options = buildPipelineOptions();
+ Pipeline p = Pipeline.create(options);
+ TestTransform transform = new TestTransform();
+
+ p.apply(Create.of(Arrays.asList(1, 2, 3)).withCoder(BigEndianIntegerCoder.of()))
+ .apply(transform);
+
+ DataflowPipelineTranslator translator = DataflowPipelineRunner
+ .fromOptions(options).getTranslator();
+
+ DataflowPipelineTranslator.registerTransformTranslator(
+ TestTransform.class,
+ new DataflowPipelineTranslator.TransformTranslator<TestTransform>() {
+ @SuppressWarnings("unchecked")
+ @Override
+ public void translate(
+ TestTransform transform,
+ DataflowPipelineTranslator.TranslationContext context) {
+ transform.translated = true;
+
+ // Note: This is about the minimum needed to fake out a
+ // translation. This obviously isn't a real translation.
+ context.addStep(transform, "TestTranslate");
+ context.addOutput("output", context.getOutput(transform));
+ }
+ });
+
+ translator.translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
+ assertTrue(transform.translated);
+ }
+
+ /** Records all the composite transforms visited within the Pipeline. */
+ private static class CompositeTransformRecorder implements PipelineVisitor {
+ private List<PTransform<?, ?>> transforms = new ArrayList<>();
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ if (node.getTransform() != null) {
+ transforms.add(node.getTransform());
+ }
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {
+ }
+
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ }
+
+ public List<PTransform<?, ?>> getCompositeTransforms() {
+ return transforms;
+ }
+ }
+
+ @Test
+ public void testApplyIsScopedToExactClass() throws IOException {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ Pipeline p = Pipeline.create(options);
+
+ Create.TimestampedValues<String> transform =
+ Create.timestamped(Arrays.asList(TimestampedValue.of("TestString", Instant.now())));
+ p.apply(transform);
+
+ CompositeTransformRecorder recorder = new CompositeTransformRecorder();
+ p.traverseTopologically(recorder);
+
+ // The recorder will also have seen a Create.Values composite as well, but we can't obtain that
+ // transform.
+ assertThat(
+ "Expected to have seen CreateTimestamped composite transform.",
+ recorder.getCompositeTransforms(),
+ hasItem(transform));
+ assertThat(
+ "Expected to have two composites, CreateTimestamped and Create.Values",
+ recorder.getCompositeTransforms(),
+ hasItem(Matchers.<PTransform<?, ?>>isA((Class) Create.Values.class)));
+ }
+
+ @Test
+ public void testToString() {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setJobName("TestJobName");
+ options.setProject("test-project");
+ options.setTempLocation("gs://test/temp/location");
+ options.setGcpCredential(new TestCredential());
+ options.setPathValidatorClass(NoopPathValidator.class);
+ assertEquals(
+ "DataflowPipelineRunner#testjobname",
+ DataflowPipelineRunner.fromOptions(options).toString());
+ }
+
+ private static PipelineOptions makeOptions(boolean streaming) {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setStreaming(streaming);
+ options.setJobName("TestJobName");
+ options.setProject("test-project");
+ options.setTempLocation("gs://test/temp/location");
+ options.setGcpCredential(new TestCredential());
+ options.setPathValidatorClass(NoopPathValidator.class);
+ return options;
+ }
+
+ private void testUnsupportedSource(PTransform<PInput, ?> source, String name, boolean streaming)
+ throws Exception {
+ String mode = streaming ? "streaming" : "batch";
+ thrown.expect(UnsupportedOperationException.class);
+ thrown.expectMessage(
+ "The DataflowPipelineRunner in " + mode + " mode does not support " + name);
+
+ Pipeline p = Pipeline.create(makeOptions(streaming));
+ p.apply(source);
+ p.run();
+ }
+
+ @Test
+ public void testBoundedSourceUnsupportedInStreaming() throws Exception {
+ testUnsupportedSource(
+ AvroSource.readFromFileWithClass("foo", String.class), "Read.Bounded", true);
+ }
+
+ @Test
+ public void testBigQueryIOSourceUnsupportedInStreaming() throws Exception {
+ testUnsupportedSource(
+ BigQueryIO.Read.from("project:bar.baz").withoutValidation(), "BigQueryIO.Read", true);
+ }
+
+ @Test
+ public void testAvroIOSourceUnsupportedInStreaming() throws Exception {
+ testUnsupportedSource(
+ AvroIO.Read.from("foo"), "AvroIO.Read", true);
+ }
+
+ @Test
+ public void testTextIOSourceUnsupportedInStreaming() throws Exception {
+ testUnsupportedSource(TextIO.Read.from("foo"), "TextIO.Read", true);
+ }
+
+ @Test
+ public void testReadBoundedSourceUnsupportedInStreaming() throws Exception {
+ testUnsupportedSource(Read.from(AvroSource.from("/tmp/test")), "Read.Bounded", true);
+ }
+
+ @Test
+ public void testReadUnboundedUnsupportedInBatch() throws Exception {
+ testUnsupportedSource(Read.from(new TestCountingSource(1)), "Read.Unbounded", false);
+ }
+
+ private void testUnsupportedSink(
+ PTransform<PCollection<String>, PDone> sink, String name, boolean streaming)
+ throws Exception {
+ thrown.expect(UnsupportedOperationException.class);
+ thrown.expectMessage(
+ "The DataflowPipelineRunner in streaming mode does not support " + name);
+
+ Pipeline p = Pipeline.create(makeOptions(streaming));
+ p.apply(Create.of("foo")).apply(sink);
+ p.run();
+ }
+
+ @Test
+ public void testAvroIOSinkUnsupportedInStreaming() throws Exception {
+ testUnsupportedSink(AvroIO.Write.to("foo").withSchema(String.class), "AvroIO.Write", true);
+ }
+
+ @Test
+ public void testTextIOSinkUnsupportedInStreaming() throws Exception {
+ testUnsupportedSink(TextIO.Write.to("foo"), "TextIO.Write", true);
+ }
+
+ @Test
+ public void testBatchViewAsListToIsmRecordForGlobalWindow() throws Exception {
+ DoFnTester<String, IsmRecord<WindowedValue<String>>> doFnTester =
+ DoFnTester.of(new BatchViewAsList.ToIsmRecordForGlobalWindowDoFn<String>());
+
+ // The order of the output elements is important relative to processing order
+ assertThat(doFnTester.processBatch(ImmutableList.of("a", "b", "c")), contains(
+ IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 0L), valueInGlobalWindow("a")),
+ IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 1L), valueInGlobalWindow("b")),
+ IsmRecord.of(ImmutableList.of(GlobalWindow.INSTANCE, 2L), valueInGlobalWindow("c"))));
+ }
+
+ @Test
+ public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception {
+ DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>,
+ IsmRecord<WindowedValue<Long>>> doFnTester =
+ DoFnTester.of(
+ new BatchViewAsList.ToIsmRecordForNonGlobalWindowDoFn<Long, IntervalWindow>(
+ IntervalWindow.getCoder()));
+
+ IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+ IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+ IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+ Iterable<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<Long>>>>> inputElements =
+ ImmutableList.of(
+ KV.of(1, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
+ KV.of(
+ windowA, WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+ KV.of(
+ windowA, WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+ KV.of(
+ windowA, WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
+ KV.of(
+ windowB, WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
+ KV.of(
+ windowB, WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING))
+ )),
+ KV.of(2, (Iterable<KV<IntervalWindow, WindowedValue<Long>>>) ImmutableList.of(
+ KV.of(
+ windowC, WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))
+ )));
+
+ // The order of the output elements is important relative to processing order
+ assertThat(doFnTester.processBatch(inputElements), contains(
+ IsmRecord.of(ImmutableList.of(windowA, 0L),
+ WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+ IsmRecord.of(ImmutableList.of(windowA, 1L),
+ WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+ IsmRecord.of(ImmutableList.of(windowA, 2L),
+ WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)),
+ IsmRecord.of(ImmutableList.of(windowB, 0L),
+ WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
+ IsmRecord.of(ImmutableList.of(windowB, 1L),
+ WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)),
+ IsmRecord.of(ImmutableList.of(windowC, 0L),
+ WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING))));
+ }
+
+ @Test
+ public void testToIsmRecordForMapLikeDoFn() throws Exception {
+ TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+ TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+ Coder<Long> keyCoder = VarLongCoder.of();
+ Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+ IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+ 1,
+ 2,
+ ImmutableList.<Coder<?>>of(
+ MetadataKeyCoder.of(keyCoder),
+ IntervalWindow.getCoder(),
+ BigEndianLongCoder.of()),
+ FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+ DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
+ IsmRecord<WindowedValue<Long>>> doFnTester =
+ DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
+ outputForSizeTag,
+ outputForEntrySetTag,
+ windowCoder,
+ keyCoder,
+ ismCoder,
+ false /* unique keys */));
+ doFnTester.setSideOutputTags(TupleTagList.of(
+ ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+ IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+ IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+ IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+ Iterable<KV<Integer,
+ Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
+ ImmutableList.of(
+ KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
+ KV.of(KV.of(1L, windowA),
+ WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+ // same window same key as to previous
+ KV.of(KV.of(1L, windowA),
+ WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
+ // same window different key as to previous
+ KV.of(KV.of(2L, windowA),
+ WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+ // different window same key as to previous
+ KV.of(KV.of(2L, windowB),
+ WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
+ // different window and different key as to previous
+ KV.of(KV.of(3L, windowB),
+ WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)))),
+ KV.of(2, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
+ // different shard
+ KV.of(KV.of(4L, windowC),
+ WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))));
+
+ // The order of the output elements is important relative to processing order
+ assertThat(doFnTester.processBatch(inputElements), contains(
+ IsmRecord.of(
+ ImmutableList.of(1L, windowA, 0L),
+ WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+ IsmRecord.of(
+ ImmutableList.of(1L, windowA, 1L),
+ WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)),
+ IsmRecord.of(
+ ImmutableList.of(2L, windowA, 0L),
+ WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)),
+ IsmRecord.of(
+ ImmutableList.of(2L, windowB, 0L),
+ WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)),
+ IsmRecord.of(
+ ImmutableList.of(3L, windowB, 0L),
+ WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)),
+ IsmRecord.of(
+ ImmutableList.of(4L, windowC, 0L),
+ WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING))));
+
+ // Verify the number of unique keys per window.
+ assertThat(doFnTester.takeSideOutputElements(outputForSizeTag), contains(
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
+ KV.of(windowA, 2L)),
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+ KV.of(windowB, 2L)),
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
+ KV.of(windowC, 1L))
+ ));
+
+ // Verify the output for the unique keys.
+ assertThat(doFnTester.takeSideOutputElements(outputForEntrySetTag), contains(
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
+ KV.of(windowA, 1L)),
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowA)),
+ KV.of(windowA, 2L)),
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+ KV.of(windowB, 2L)),
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+ KV.of(windowB, 3L)),
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowC)),
+ KV.of(windowC, 4L))
+ ));
+ }
+
+ @Test
+ public void testToIsmRecordForMapLikeDoFnWithoutUniqueKeysThrowsException() throws Exception {
+ TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+ TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+ Coder<Long> keyCoder = VarLongCoder.of();
+ Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+ IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+ 1,
+ 2,
+ ImmutableList.<Coder<?>>of(
+ MetadataKeyCoder.of(keyCoder),
+ IntervalWindow.getCoder(),
+ BigEndianLongCoder.of()),
+ FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+ DoFnTester<KV<Integer, Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>,
+ IsmRecord<WindowedValue<Long>>> doFnTester =
+ DoFnTester.of(new BatchViewAsMultimap.ToIsmRecordForMapLikeDoFn<Long, Long, IntervalWindow>(
+ outputForSizeTag,
+ outputForEntrySetTag,
+ windowCoder,
+ keyCoder,
+ ismCoder,
+ true /* unique keys */));
+ doFnTester.setSideOutputTags(TupleTagList.of(
+ ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+ IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+
+ Iterable<KV<Integer,
+ Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>>> inputElements =
+ ImmutableList.of(
+ KV.of(1, (Iterable<KV<KV<Long, IntervalWindow>, WindowedValue<Long>>>) ImmutableList.of(
+ KV.of(KV.of(1L, windowA),
+ WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)),
+ // same window same key as to previous
+ KV.of(KV.of(1L, windowA),
+ WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)))));
+
+ try {
+ doFnTester.processBatch(inputElements);
+ fail("Expected UserCodeException");
+ } catch (UserCodeException e) {
+ assertTrue(e.getCause() instanceof IllegalStateException);
+ IllegalStateException rootCause = (IllegalStateException) e.getCause();
+ assertThat(rootCause.getMessage(), containsString("Unique keys are expected but found key"));
+ }
+ }
+
+ @Test
+ public void testToIsmMetadataRecordForSizeDoFn() throws Exception {
+ TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+ TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+ Coder<Long> keyCoder = VarLongCoder.of();
+ Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+ IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+ 1,
+ 2,
+ ImmutableList.<Coder<?>>of(
+ MetadataKeyCoder.of(keyCoder),
+ IntervalWindow.getCoder(),
+ BigEndianLongCoder.of()),
+ FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+ DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
+ IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
+ new BatchViewAsMultimap.ToIsmMetadataRecordForSizeDoFn<Long, Long, IntervalWindow>(
+ windowCoder));
+ doFnTester.setSideOutputTags(TupleTagList.of(
+ ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+ IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+ IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+ IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+ Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
+ ImmutableList.of(
+ KV.of(1,
+ (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+ KV.of(windowA, 2L),
+ KV.of(windowA, 3L),
+ KV.of(windowB, 7L))),
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+ (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+ KV.of(windowC, 9L))));
+
+ // The order of the output elements is important relative to processing order
+ assertThat(doFnTester.processBatch(inputElements), contains(
+ IsmRecord.<WindowedValue<Long>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 0L),
+ CoderUtils.encodeToByteArray(VarLongCoder.of(), 5L)),
+ IsmRecord.<WindowedValue<Long>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 0L),
+ CoderUtils.encodeToByteArray(VarLongCoder.of(), 7L)),
+ IsmRecord.<WindowedValue<Long>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 0L),
+ CoderUtils.encodeToByteArray(VarLongCoder.of(), 9L))
+ ));
+ }
+
+ @Test
+ public void testToIsmMetadataRecordForKeyDoFn() throws Exception {
+ TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForSizeTag = new TupleTag<>();
+ TupleTag<KV<Integer, KV<IntervalWindow, Long>>> outputForEntrySetTag = new TupleTag<>();
+
+ Coder<Long> keyCoder = VarLongCoder.of();
+ Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+ IsmRecordCoder<WindowedValue<Long>> ismCoder = IsmRecordCoder.of(
+ 1,
+ 2,
+ ImmutableList.<Coder<?>>of(
+ MetadataKeyCoder.of(keyCoder),
+ IntervalWindow.getCoder(),
+ BigEndianLongCoder.of()),
+ FullWindowedValueCoder.of(VarLongCoder.of(), windowCoder));
+
+ DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, Long>>>,
+ IsmRecord<WindowedValue<Long>>> doFnTester = DoFnTester.of(
+ new BatchViewAsMultimap.ToIsmMetadataRecordForKeyDoFn<Long, Long, IntervalWindow>(
+ keyCoder, windowCoder));
+ doFnTester.setSideOutputTags(TupleTagList.of(
+ ImmutableList.<TupleTag<?>>of(outputForSizeTag, outputForEntrySetTag)));
+
+ IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+ IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+ IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+ Iterable<KV<Integer, Iterable<KV<IntervalWindow, Long>>>> inputElements =
+ ImmutableList.of(
+ KV.of(1,
+ (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+ KV.of(windowA, 2L),
+ // same window as previous
+ KV.of(windowA, 3L),
+ // different window as previous
+ KV.of(windowB, 3L))),
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(), windowB)),
+ (Iterable<KV<IntervalWindow, Long>>) ImmutableList.of(
+ KV.of(windowC, 3L))));
+
+ // The order of the output elements is important relative to processing order
+ assertThat(doFnTester.processBatch(inputElements), contains(
+ IsmRecord.<WindowedValue<Long>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 1L),
+ CoderUtils.encodeToByteArray(VarLongCoder.of(), 2L)),
+ IsmRecord.<WindowedValue<Long>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), windowA, 2L),
+ CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
+ IsmRecord.<WindowedValue<Long>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), windowB, 1L),
+ CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L)),
+ IsmRecord.<WindowedValue<Long>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), windowC, 1L),
+ CoderUtils.encodeToByteArray(VarLongCoder.of(), 3L))
+ ));
+ }
+
+ @Test
+ public void testToMapDoFn() throws Exception {
+ Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+ DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
+ IsmRecord<WindowedValue<TransformedMap<Long,
+ WindowedValue<Long>,
+ Long>>>> doFnTester =
+ DoFnTester.of(new BatchViewAsMap.ToMapDoFn<Long, Long, IntervalWindow>(windowCoder));
+
+
+ IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+ IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+ IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+ Iterable<KV<Integer,
+ Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
+ ImmutableList.of(
+ KV.of(1,
+ (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+ KV.of(windowA, WindowedValue.of(
+ KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
+ KV.of(windowA, WindowedValue.of(
+ KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
+ KV.of(windowB, WindowedValue.of(
+ KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
+ KV.of(windowB, WindowedValue.of(
+ KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
+ KV.of(2,
+ (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+ KV.of(windowC, WindowedValue.of(
+ KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
+
+ // The order of the output elements is important relative to processing order
+ List<IsmRecord<WindowedValue<TransformedMap<Long,
+ WindowedValue<Long>,
+ Long>>>> output =
+ doFnTester.processBatch(inputElements);
+ assertEquals(3, output.size());
+ Map<Long, Long> outputMap;
+
+ outputMap = output.get(0).getValue().getValue();
+ assertEquals(2, outputMap.size());
+ assertEquals(ImmutableMap.of(1L, 11L, 2L, 21L), outputMap);
+
+ outputMap = output.get(1).getValue().getValue();
+ assertEquals(2, outputMap.size());
+ assertEquals(ImmutableMap.of(2L, 21L, 3L, 31L), outputMap);
+
+ outputMap = output.get(2).getValue().getValue();
+ assertEquals(1, outputMap.size());
+ assertEquals(ImmutableMap.of(4L, 41L), outputMap);
+ }
+
+ @Test
+ public void testToMultimapDoFn() throws Exception {
+ Coder<IntervalWindow> windowCoder = IntervalWindow.getCoder();
+
+ DoFnTester<KV<Integer, Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>,
+ IsmRecord<WindowedValue<TransformedMap<Long,
+ Iterable<WindowedValue<Long>>,
+ Iterable<Long>>>>> doFnTester =
+ DoFnTester.of(
+ new BatchViewAsMultimap.ToMultimapDoFn<Long, Long, IntervalWindow>(windowCoder));
+
+
+ IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10));
+ IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20));
+ IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30));
+
+ Iterable<KV<Integer,
+ Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>>> inputElements =
+ ImmutableList.of(
+ KV.of(1,
+ (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+ KV.of(windowA, WindowedValue.of(
+ KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)),
+ KV.of(windowA, WindowedValue.of(
+ KV.of(1L, 12L), new Instant(5), windowA, PaneInfo.NO_FIRING)),
+ KV.of(windowA, WindowedValue.of(
+ KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)),
+ KV.of(windowB, WindowedValue.of(
+ KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)),
+ KV.of(windowB, WindowedValue.of(
+ KV.of(3L, 31L), new Instant(15), windowB, PaneInfo.NO_FIRING)))),
+ KV.of(2,
+ (Iterable<KV<IntervalWindow, WindowedValue<KV<Long, Long>>>>) ImmutableList.of(
+ KV.of(windowC, WindowedValue.of(
+ KV.of(4L, 41L), new Instant(25), windowC, PaneInfo.NO_FIRING)))));
+
+ // The order of the output elements is important relative to processing order
+ List<IsmRecord<WindowedValue<TransformedMap<Long,
+ Iterable<WindowedValue<Long>>,
+ Iterable<Long>>>>> output =
+ doFnTester.processBatch(inputElements);
+ assertEquals(3, output.size());
+ Map<Long, Iterable<Long>> outputMap;
+
+ outputMap = output.get(0).getValue().getValue();
+ assertEquals(2, outputMap.size());
+ assertThat(outputMap.get(1L), containsInAnyOrder(11L, 12L));
+ assertThat(outputMap.get(2L), containsInAnyOrder(21L));
+
+ outputMap = output.get(1).getValue().getValue();
+ assertEquals(2, outputMap.size());
+ assertThat(outputMap.get(2L), containsInAnyOrder(21L));
+ assertThat(outputMap.get(3L), containsInAnyOrder(31L));
+
+ outputMap = output.get(2).getValue().getValue();
+ assertEquals(1, outputMap.size());
+ assertThat(outputMap.get(4L), containsInAnyOrder(41L));
+ }
+}
[17/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
Reorganize Java packages in the sources of the Google Cloud Dataflow runner
Packages are moving from org.apache.beam.sdk to org.apache.beam.runners.dataflow.
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/02190985
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/02190985
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/02190985
Branch: refs/heads/master
Commit: 021909855fcf6729ce6ccb9b9ff76f1ca5af35db
Parents: 9e19efd
Author: Davor Bonaci <da...@google.com>
Authored: Mon Apr 25 14:16:03 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Apr 26 17:59:39 2016 -0700
----------------------------------------------------------------------
.../apache/beam/examples/MinimalWordCount.java | 4 +-
.../org/apache/beam/examples/WordCount.java | 2 +-
.../examples/common/DataflowExampleOptions.java | 2 +-
.../examples/common/DataflowExampleUtils.java | 8 +-
.../common/ExampleBigQueryTableOptions.java | 2 +-
...xamplePubsubTopicAndSubscriptionOptions.java | 2 +-
.../common/ExamplePubsubTopicOptions.java | 2 +-
.../examples/common/PubsubFileInjector.java | 2 +-
.../beam/examples/complete/AutoComplete.java | 2 +-
.../examples/complete/StreamingWordExtract.java | 2 +-
.../examples/complete/TopWikipediaSessions.java | 2 +-
.../beam/examples/cookbook/DeDupExample.java | 2 +-
.../beam/examples/cookbook/TriggerExample.java | 4 +-
.../org/apache/beam/examples/WordCountIT.java | 4 +-
.../beam/examples/MinimalWordCountJava8.java | 4 +-
.../beam/examples/complete/game/GameStats.java | 2 +-
.../examples/complete/game/LeaderBoard.java | 2 +-
.../beam/runners/flink/examples/TFIDF.java | 1 +
.../beam/runners/flink/examples/WordCount.java | 9 +-
.../flink/examples/streaming/AutoComplete.java | 21 +-
.../flink/examples/streaming/JoinExamples.java | 6 +-
.../KafkaWindowedWordCountExample.java | 11 +-
.../examples/streaming/WindowedWordCount.java | 13 +-
.../runners/flink/FlinkPipelineOptions.java | 2 +-
.../beam/runners/flink/FlinkPipelineRunner.java | 4 +-
.../FlinkBatchPipelineTranslator.java | 2 +-
.../FlinkStreamingPipelineTranslator.java | 2 +-
runners/google-cloud-dataflow-java/pom.xml | 4 +-
.../BlockingDataflowPipelineRunner.java | 186 ++
.../DataflowJobAlreadyExistsException.java | 35 +
.../DataflowJobAlreadyUpdatedException.java | 34 +
.../dataflow/DataflowJobCancelledException.java | 39 +
.../runners/dataflow/DataflowJobException.java | 41 +
.../dataflow/DataflowJobExecutionException.java | 35 +
.../dataflow/DataflowJobUpdatedException.java | 51 +
.../runners/dataflow/DataflowPipelineJob.java | 397 +++
.../dataflow/DataflowPipelineRegistrar.java | 62 +
.../dataflow/DataflowPipelineRunner.java | 3025 ++++++++++++++++++
.../dataflow/DataflowPipelineRunnerHooks.java | 39 +
.../dataflow/DataflowPipelineTranslator.java | 1059 ++++++
.../dataflow/DataflowServiceException.java | 33 +
.../dataflow/internal/AssignWindows.java | 89 +
.../dataflow/internal/BigQueryIOTranslator.java | 72 +
.../dataflow/internal/CustomSources.java | 121 +
.../internal/DataflowAggregatorTransforms.java | 81 +
.../internal/DataflowMetricUpdateExtractor.java | 111 +
.../dataflow/internal/PubsubIOTranslator.java | 108 +
.../dataflow/internal/ReadTranslator.java | 105 +
.../runners/dataflow/internal/package-info.java | 21 +
.../BlockingDataflowPipelineOptions.java | 55 +
.../dataflow/options/CloudDebuggerOptions.java | 56 +
.../options/DataflowPipelineDebugOptions.java | 247 ++
.../options/DataflowPipelineOptions.java | 126 +
.../DataflowPipelineWorkerPoolOptions.java | 263 ++
.../options/DataflowProfilingOptions.java | 50 +
.../options/DataflowWorkerHarnessOptions.java | 55 +
.../options/DataflowWorkerLoggingOptions.java | 159 +
.../testing/TestDataflowPipelineOptions.java | 30 +
.../testing/TestDataflowPipelineRunner.java | 273 ++
.../dataflow/util/DataflowPathValidator.java | 100 +
.../dataflow/util/DataflowTransport.java | 114 +
.../beam/runners/dataflow/util/GcsStager.java | 55 +
.../runners/dataflow/util/MonitoringUtil.java | 237 ++
.../beam/runners/dataflow/util/PackageUtil.java | 333 ++
.../beam/runners/dataflow/util/Stager.java | 30 +
.../BlockingDataflowPipelineOptions.java | 50 -
.../beam/sdk/options/CloudDebuggerOptions.java | 53 -
.../options/DataflowPipelineDebugOptions.java | 242 --
.../sdk/options/DataflowPipelineOptions.java | 115 -
.../DataflowPipelineWorkerPoolOptions.java | 258 --
.../sdk/options/DataflowProfilingOptions.java | 48 -
.../options/DataflowWorkerHarnessOptions.java | 51 -
.../options/DataflowWorkerLoggingOptions.java | 155 -
.../runners/BlockingDataflowPipelineRunner.java | 185 --
.../DataflowJobAlreadyExistsException.java | 35 -
.../DataflowJobAlreadyUpdatedException.java | 34 -
.../runners/DataflowJobCancelledException.java | 39 -
.../beam/sdk/runners/DataflowJobException.java | 41 -
.../runners/DataflowJobExecutionException.java | 35 -
.../runners/DataflowJobUpdatedException.java | 51 -
.../beam/sdk/runners/DataflowPipelineJob.java | 395 ---
.../sdk/runners/DataflowPipelineRegistrar.java | 60 -
.../sdk/runners/DataflowPipelineRunner.java | 3022 -----------------
.../runners/DataflowPipelineRunnerHooks.java | 39 -
.../sdk/runners/DataflowPipelineTranslator.java | 1058 ------
.../sdk/runners/DataflowServiceException.java | 33 -
.../sdk/runners/dataflow/AssignWindows.java | 89 -
.../runners/dataflow/BigQueryIOTranslator.java | 72 -
.../sdk/runners/dataflow/CustomSources.java | 121 -
.../dataflow/DataflowAggregatorTransforms.java | 81 -
.../dataflow/DataflowMetricUpdateExtractor.java | 111 -
.../runners/dataflow/PubsubIOTranslator.java | 108 -
.../sdk/runners/dataflow/ReadTranslator.java | 105 -
.../beam/sdk/runners/dataflow/package-info.java | 21 -
.../testing/TestDataflowPipelineOptions.java | 28 -
.../sdk/testing/TestDataflowPipelineRunner.java | 271 --
.../beam/sdk/util/DataflowPathValidator.java | 99 -
.../apache/beam/sdk/util/DataflowTransport.java | 113 -
.../org/apache/beam/sdk/util/GcsStager.java | 55 -
.../apache/beam/sdk/util/MonitoringUtil.java | 236 --
.../org/apache/beam/sdk/util/PackageUtil.java | 328 --
.../java/org/apache/beam/sdk/util/Stager.java | 30 -
.../BlockingDataflowPipelineRunnerTest.java | 302 ++
.../dataflow/DataflowPipelineJobTest.java | 608 ++++
.../dataflow/DataflowPipelineRegistrarTest.java | 75 +
.../dataflow/DataflowPipelineRunnerTest.java | 1401 ++++++++
.../DataflowPipelineTranslatorTest.java | 967 ++++++
.../dataflow/internal/CustomSourcesTest.java | 276 ++
.../runners/dataflow/io/DataflowTextIOTest.java | 119 +
.../DataflowPipelineDebugOptionsTest.java | 43 +
.../options/DataflowPipelineOptionsTest.java | 93 +
.../options/DataflowProfilingOptionsTest.java | 51 +
.../DataflowWorkerLoggingOptionsTest.java | 77 +
.../testing/TestDataflowPipelineRunnerTest.java | 381 +++
.../transforms/DataflowGroupByKeyTest.java | 113 +
.../dataflow/transforms/DataflowViewTest.java | 208 ++
.../util/DataflowPathValidatorTest.java | 94 +
.../dataflow/util/MonitoringUtilTest.java | 151 +
.../runners/dataflow/util/PackageUtilTest.java | 486 +++
.../apache/beam/sdk/io/DataflowTextIOTest.java | 118 -
.../DataflowPipelineDebugOptionsTest.java | 41 -
.../options/DataflowPipelineOptionsTest.java | 92 -
.../options/DataflowProfilingOptionsTest.java | 49 -
.../DataflowWorkerLoggingOptionsTest.java | 77 -
.../BlockingDataflowPipelineRunnerTest.java | 302 --
.../sdk/runners/DataflowPipelineJobTest.java | 606 ----
.../runners/DataflowPipelineRegistrarTest.java | 74 -
.../sdk/runners/DataflowPipelineRunnerTest.java | 1400 --------
.../runners/DataflowPipelineTranslatorTest.java | 965 ------
.../sdk/runners/dataflow/CustomSourcesTest.java | 276 --
.../testing/TestDataflowPipelineRunnerTest.java | 379 ---
.../sdk/transforms/DataflowGroupByKeyTest.java | 110 -
.../beam/sdk/transforms/DataflowViewTest.java | 205 --
.../sdk/util/DataflowPathValidatorTest.java | 92 -
.../beam/sdk/util/MonitoringUtilTest.java | 149 -
.../apache/beam/sdk/util/PackageUtilTest.java | 484 ---
.../apache/beam/sdk/testing/TestPipeline.java | 2 +-
.../src/main/java/MinimalWordCount.java | 4 +-
.../src/main/java/WindowedWordCount.java | 10 +-
.../src/main/java/WordCount.java | 2 +-
.../java/common/DataflowExampleOptions.java | 2 +-
.../main/java/common/DataflowExampleUtils.java | 9 +-
.../common/ExampleBigQueryTableOptions.java | 5 +-
.../java/common/ExamplePubsubTopicOptions.java | 2 +-
.../main/java/common/PubsubFileInjector.java | 9 +-
145 files changed, 13479 insertions(+), 13347 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
----------------------------------------------------------------------
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 6c33ca6..ec517c4 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
@@ -17,11 +17,11 @@
*/
package org.apache.beam.examples;
+import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
+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.DataflowPipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.BlockingDataflowPipelineRunner;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.MapElements;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
----------------------------------------------------------------------
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 9fb9856..364d075 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,9 +17,9 @@
*/
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.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.DefaultValueFactory;
import org.apache.beam.sdk.options.Description;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
index 5c62146..2e8ef3d 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleOptions.java
@@ -17,7 +17,7 @@
*/
package org.apache.beam.examples.common;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/common/DataflowExampleUtils.java
----------------------------------------------------------------------
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 93c2358..7ac71d3 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
@@ -17,18 +17,18 @@
*/
package org.apache.beam.examples.common;
+import org.apache.beam.runners.dataflow.DataflowPipelineJob;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.options.BigQueryOptions;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.runners.DataflowPipelineJob;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
import org.apache.beam.sdk.runners.DirectPipelineRunner;
import org.apache.beam.sdk.transforms.IntraBundleParallelization;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
-import org.apache.beam.sdk.util.MonitoringUtil;
import org.apache.beam.sdk.util.Transport;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
index 647d508..36304a8 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java
@@ -17,7 +17,7 @@
*/
package org.apache.beam.examples.common;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.DefaultValueFactory;
import org.apache.beam.sdk.options.Description;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
index b3da88d..22bcf4e 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java
@@ -17,7 +17,7 @@
*/
package org.apache.beam.examples.common;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.DefaultValueFactory;
import org.apache.beam.sdk.options.Description;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
index 17f5b28..603e309 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java
@@ -17,7 +17,7 @@
*/
package org.apache.beam.examples.common;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.DefaultValueFactory;
import org.apache.beam.sdk.options.Description;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
index b71b6f3..82d58b6 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java
@@ -17,9 +17,9 @@
*/
package org.apache.beam.examples.common;
+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.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
----------------------------------------------------------------------
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 e1b658b..2f2283c 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
@@ -20,6 +20,7 @@ package org.apache.beam.examples.complete;
import org.apache.beam.examples.common.DataflowExampleUtils;
import org.apache.beam.examples.common.ExampleBigQueryTableOptions;
import org.apache.beam.examples.common.ExamplePubsubTopicOptions;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.coders.AvroCoder;
@@ -31,7 +32,6 @@ import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.Filter;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
index 7107e34..5de08da 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
@@ -20,6 +20,7 @@ package org.apache.beam.examples.complete;
import org.apache.beam.examples.common.DataflowExampleUtils;
import org.apache.beam.examples.common.ExampleBigQueryTableOptions;
import org.apache.beam.examples.common.ExamplePubsubTopicOptions;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.BigQueryIO;
@@ -27,7 +28,6 @@ import org.apache.beam.sdk.io.PubsubIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ParDo;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
----------------------------------------------------------------------
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 fa587ea..e7090fd 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
@@ -17,10 +17,10 @@
*/
package org.apache.beam.examples.complete;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.TableRowJsonCoder;
import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
----------------------------------------------------------------------
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 1d34245..fe2bbc8 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,9 +17,9 @@
*/
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.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.DefaultValueFactory;
import org.apache.beam.sdk.options.Description;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
index 72976e3..28885a7 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
@@ -22,16 +22,16 @@ import org.apache.beam.examples.common.DataflowExampleUtils;
import org.apache.beam.examples.common.ExampleBigQueryTableOptions;
import org.apache.beam.examples.common.ExamplePubsubTopicOptions;
import org.apache.beam.examples.common.PubsubFileInjector;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.BigQueryIO;
import org.apache.beam.sdk.io.PubsubIO;
import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess;
import org.apache.beam.sdk.transforms.GroupByKey;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java
----------------------------------------------------------------------
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 a5ad707..56ca98c 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
@@ -22,10 +22,10 @@ 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.DataflowPipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.TestDataflowPipelineRunner;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.TestPipelineOptions;
import com.google.common.base.Joiner;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
index 493344e..398d517 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
@@ -17,11 +17,11 @@
*/
package org.apache.beam.examples;
+import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
+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.DataflowPipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.BlockingDataflowPipelineRunner;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.transforms.Filter;
import org.apache.beam.sdk.transforms.FlatMapElements;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
----------------------------------------------------------------------
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 2d14264..12dfdf9 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
@@ -19,13 +19,13 @@ package org.apache.beam.examples.complete.game;
import org.apache.beam.examples.common.DataflowExampleUtils;
import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.PubsubIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
import org.apache.beam.sdk.transforms.Aggregator;
import org.apache.beam.sdk.transforms.Combine;
import org.apache.beam.sdk.transforms.DoFn;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
----------------------------------------------------------------------
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 97958b0..a5d9fb9 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
@@ -21,6 +21,7 @@ import org.apache.beam.examples.common.DataflowExampleOptions;
import org.apache.beam.examples.common.DataflowExampleUtils;
import org.apache.beam.examples.complete.game.utils.WriteToBigQuery;
import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.PubsubIO;
@@ -28,7 +29,6 @@ import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.Validation;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
index 3eb1327..0afde0a 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
@@ -53,6 +53,7 @@ import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.PDone;
import org.apache.beam.sdk.values.PInput;
import org.apache.beam.sdk.values.TupleTag;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
index 04e11c1..702fb63 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
@@ -25,7 +25,14 @@ import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.Sum;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
index 2bc3490..9d1168b 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
@@ -22,17 +22,32 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.io.Unbounded
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.AvroCoder;
import org.apache.beam.sdk.coders.DefaultCoder;
-import org.apache.beam.sdk.io.*;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.*;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Filter;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Partition;
import org.apache.beam.sdk.transforms.Partition.PartitionFn;
-import org.apache.beam.sdk.transforms.windowing.*;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.Top;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionList;
+
import org.joda.time.Duration;
import java.io.IOException;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
index 8c8490b..d3e963d 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
@@ -29,11 +29,15 @@ 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.transforms.windowing.*;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TupleTag;
+
import org.joda.time.Duration;
/**
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
index 95210ad..abb9fea 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
@@ -25,10 +25,17 @@ import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.*;
-import org.apache.beam.sdk.transforms.windowing.*;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
+
import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08;
import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
import org.joda.time.Duration;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
index 9f77f8c..e803e6e 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
@@ -20,12 +20,19 @@ package org.apache.beam.runners.flink.examples.streaming;
import org.apache.beam.runners.flink.FlinkPipelineRunner;
import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.*;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.transforms.*;
-import org.apache.beam.sdk.transforms.windowing.*;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
index bfb0d6a..8c82abd 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
@@ -18,8 +18,8 @@
package org.apache.beam.runners.flink;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.ApplicationNameOptions;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
----------------------------------------------------------------------
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 bb016fa..a389d7a 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
@@ -17,11 +17,11 @@
*/
package org.apache.beam.runners.flink;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
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.DataflowPipelineRunner;
import org.apache.beam.sdk.runners.PipelineRunner;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.values.PInput;
@@ -46,7 +46,7 @@ import java.util.Map;
* pipeline by first translating them to a Flink Plan and then executing them either locally
* or on a Flink cluster, depending on the configuration.
* <p>
- * This is based on {@link org.apache.beam.sdk.runners.DataflowPipelineRunner}.
+ * This is based on {@link org.apache.beam.runners.dataflow.DataflowPipelineRunner}.
*/
public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
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 5ce828c..456cf09 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
@@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory;
/**
* FlinkBatchPipelineTranslator knows how to translate Pipeline objects into Flink Jobs.
- * This is based on {@link org.apache.beam.sdk.runners.DataflowPipelineTranslator}
+ * This is based on {@link org.apache.beam.runners.dataflow.DataflowPipelineTranslator}
*/
public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
index 4359842..ebaf6ba 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
@@ -32,7 +32,7 @@ import org.slf4j.LoggerFactory;
* {@link org.apache.beam.sdk.values.PCollection}-based job into a
* {@link org.apache.flink.streaming.api.datastream.DataStream} one.
*
- * This is based on {@link org.apache.beam.sdk.runners.DataflowPipelineTranslator}
+ * This is based on {@link org.apache.beam.runners.dataflow.DataflowPipelineTranslator}
* */
public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index 7893975..beb340c 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -177,8 +177,8 @@
<windowtitle>Google Cloud Dataflow Java Runner ${project.version}</windowtitle>
<doctitle>Google Cloud Dataflow Runner for Java, version ${project.version}</doctitle>
- <subpackages>org.apache.beam.sdk</subpackages>
- <additionalparam>-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}</additionalparam>
+ <subpackages>org.apache.beam.runners.dataflow</subpackages>
+ <additionalparam>-exclude org.apache.beam.sdk.runners.dataflow.internal:org.apache.beam.sdk.runners.dataflow.testing:org.apache.beam.sdk.runners.dataflow.util ${dataflow.javadoc_opts}</additionalparam>
<use>false</use>
<quiet>true</quiet>
<bottom><![CDATA[<br>]]></bottom>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunner.java
new file mode 100644
index 0000000..d8ee16a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunner.java
@@ -0,0 +1,186 @@
+/*
+ * 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.dataflow;
+
+import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.options.PipelineOptions;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link PipelineRunner} that's like {@link DataflowPipelineRunner}
+ * but that waits for the launched job to finish.
+ *
+ * <p>Prints out job status updates and console messages while it waits.
+ *
+ * <p>Returns the final job state, or throws an exception if the job
+ * fails or cannot be monitored.
+ *
+ * <p><h3>Permissions</h3>
+ * When reading from a Dataflow source or writing to a Dataflow sink using
+ * {@code BlockingDataflowPipelineRunner}, the Google cloud services account and the Google compute
+ * engine service account of the GCP project running the Dataflow Job will need access to the
+ * corresponding source/sink.
+ *
+ * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
+ * Dataflow Security and Permissions</a> for more details.
+ */
+public class BlockingDataflowPipelineRunner extends
+ PipelineRunner<DataflowPipelineJob> {
+ private static final Logger LOG = LoggerFactory.getLogger(BlockingDataflowPipelineRunner.class);
+
+ // Defaults to an infinite wait period.
+ // TODO: make this configurable after removal of option map.
+ private static final long BUILTIN_JOB_TIMEOUT_SEC = -1L;
+
+ private final DataflowPipelineRunner dataflowPipelineRunner;
+ private final BlockingDataflowPipelineOptions options;
+
+ protected BlockingDataflowPipelineRunner(
+ DataflowPipelineRunner internalRunner,
+ BlockingDataflowPipelineOptions options) {
+ this.dataflowPipelineRunner = internalRunner;
+ this.options = options;
+ }
+
+ /**
+ * Constructs a runner from the provided options.
+ */
+ public static BlockingDataflowPipelineRunner fromOptions(
+ PipelineOptions options) {
+ BlockingDataflowPipelineOptions dataflowOptions =
+ PipelineOptionsValidator.validate(BlockingDataflowPipelineOptions.class, options);
+ DataflowPipelineRunner dataflowPipelineRunner =
+ DataflowPipelineRunner.fromOptions(dataflowOptions);
+
+ return new BlockingDataflowPipelineRunner(dataflowPipelineRunner, dataflowOptions);
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * @throws DataflowJobExecutionException if there is an exception during job execution.
+ * @throws DataflowServiceException if there is an exception retrieving information about the job.
+ */
+ @Override
+ public DataflowPipelineJob run(Pipeline p) {
+ final DataflowPipelineJob job = dataflowPipelineRunner.run(p);
+
+ // We ignore the potential race condition here (Ctrl-C after job submission but before the
+ // shutdown hook is registered). Even if we tried to do something smarter (eg., SettableFuture)
+ // the run method (which produces the job) could fail or be Ctrl-C'd before it had returned a
+ // job. The display of the command to cancel the job is best-effort anyways -- RPC's could fail,
+ // etc. If the user wants to verify the job was cancelled they should look at the job status.
+ Thread shutdownHook = new Thread() {
+ @Override
+ public void run() {
+ LOG.warn("Job is already running in Google Cloud Platform, Ctrl-C will not cancel it.\n"
+ + "To cancel the job in the cloud, run:\n> {}",
+ MonitoringUtil.getGcloudCancelCommand(options, job.getJobId()));
+ }
+ };
+
+ try {
+ Runtime.getRuntime().addShutdownHook(shutdownHook);
+
+ @Nullable
+ State result;
+ try {
+ result = job.waitToFinish(
+ BUILTIN_JOB_TIMEOUT_SEC, TimeUnit.SECONDS,
+ new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
+ } catch (IOException | InterruptedException ex) {
+ if (ex instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
+ LOG.debug("Exception caught while retrieving status for job {}", job.getJobId(), ex);
+ throw new DataflowServiceException(
+ job, "Exception caught while retrieving status for job " + job.getJobId(), ex);
+ }
+
+ if (result == null) {
+ throw new DataflowServiceException(
+ job, "Timed out while retrieving status for job " + job.getJobId());
+ }
+
+ LOG.info("Job finished with status {}", result);
+ if (!result.isTerminal()) {
+ throw new IllegalStateException("Expected terminal state for job " + job.getJobId()
+ + ", got " + result);
+ }
+
+ if (result == State.DONE) {
+ return job;
+ } else if (result == State.UPDATED) {
+ DataflowPipelineJob newJob = job.getReplacedByJob();
+ LOG.info("Job {} has been updated and is running as the new job with id {}."
+ + "To access the updated job on the Dataflow monitoring console, please navigate to {}",
+ job.getJobId(),
+ newJob.getJobId(),
+ MonitoringUtil.getJobMonitoringPageURL(newJob.getProjectId(), newJob.getJobId()));
+ throw new DataflowJobUpdatedException(
+ job,
+ String.format("Job %s updated; new job is %s.", job.getJobId(), newJob.getJobId()),
+ newJob);
+ } else if (result == State.CANCELLED) {
+ String message = String.format("Job %s cancelled by user", job.getJobId());
+ LOG.info(message);
+ throw new DataflowJobCancelledException(job, message);
+ } else {
+ throw new DataflowJobExecutionException(job, "Job " + job.getJobId()
+ + " failed with status " + result);
+ }
+ } finally {
+ Runtime.getRuntime().removeShutdownHook(shutdownHook);
+ }
+ }
+
+ @Override
+ public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+ PTransform<InputT, OutputT> transform, InputT input) {
+ return dataflowPipelineRunner.apply(transform, input);
+ }
+
+ /**
+ * Sets callbacks to invoke during execution. See {@link DataflowPipelineRunnerHooks}.
+ */
+ @Experimental
+ public void setHooks(DataflowPipelineRunnerHooks hooks) {
+ this.dataflowPipelineRunner.setHooks(hooks);
+ }
+
+ @Override
+ public String toString() {
+ return "BlockingDataflowPipelineRunner#" + options.getJobName();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobAlreadyExistsException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobAlreadyExistsException.java
new file mode 100644
index 0000000..2b73bf7
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobAlreadyExistsException.java
@@ -0,0 +1,35 @@
+/*
+ * 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.dataflow;
+
+/**
+ * An exception that is thrown if the unique job name constraint of the Dataflow
+ * service is broken because an existing job with the same job name is currently active.
+ * The {@link DataflowPipelineJob} contained within this exception contains information
+ * about the pre-existing job.
+ */
+public class DataflowJobAlreadyExistsException extends DataflowJobException {
+ /**
+ * Create a new {@code DataflowJobAlreadyExistsException} with the specified {@link
+ * DataflowPipelineJob} and message.
+ */
+ public DataflowJobAlreadyExistsException(
+ DataflowPipelineJob job, String message) {
+ super(job, message, null);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobAlreadyUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobAlreadyUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobAlreadyUpdatedException.java
new file mode 100644
index 0000000..be11637
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobAlreadyUpdatedException.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.runners.dataflow;
+
+/**
+ * An exception that is thrown if the existing job has already been updated within the Dataflow
+ * service and is no longer able to be updated. The {@link DataflowPipelineJob} contained within
+ * this exception contains information about the pre-existing updated job.
+ */
+public class DataflowJobAlreadyUpdatedException extends DataflowJobException {
+ /**
+ * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
+ * DataflowPipelineJob} and message.
+ */
+ public DataflowJobAlreadyUpdatedException(
+ DataflowPipelineJob job, String message) {
+ super(job, message, null);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobCancelledException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobCancelledException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobCancelledException.java
new file mode 100644
index 0000000..de44a19
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobCancelledException.java
@@ -0,0 +1,39 @@
+/*
+ * 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.dataflow;
+
+/**
+ * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.
+ */
+public class DataflowJobCancelledException extends DataflowJobException {
+ /**
+ * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
+ * DataflowPipelineJob} and message.
+ */
+ public DataflowJobCancelledException(DataflowPipelineJob job, String message) {
+ super(job, message, null);
+ }
+
+ /**
+ * Create a new {@code DataflowJobAlreadyUpdatedException} with the specified {@link
+ * DataflowPipelineJob}, message, and cause.
+ */
+ public DataflowJobCancelledException(DataflowPipelineJob job, String message, Throwable cause) {
+ super(job, message, cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java
new file mode 100644
index 0000000..74c0f80
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.dataflow;
+
+import java.util.Objects;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@link RuntimeException} that contains information about a {@link DataflowPipelineJob}.
+ */
+public abstract class DataflowJobException extends RuntimeException {
+ private final DataflowPipelineJob job;
+
+ DataflowJobException(DataflowPipelineJob job, String message, @Nullable Throwable cause) {
+ super(message, cause);
+ this.job = Objects.requireNonNull(job);
+ }
+
+ /**
+ * Returns the failed job.
+ */
+ public DataflowPipelineJob getJob() {
+ return job;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobExecutionException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobExecutionException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobExecutionException.java
new file mode 100644
index 0000000..11b8723
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobExecutionException.java
@@ -0,0 +1,35 @@
+/*
+ * 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.dataflow;
+
+import javax.annotation.Nullable;
+
+/**
+ * Signals that a job run by a {@link BlockingDataflowPipelineRunner} fails during execution, and
+ * provides access to the failed job.
+ */
+public class DataflowJobExecutionException extends DataflowJobException {
+ DataflowJobExecutionException(DataflowPipelineJob job, String message) {
+ this(job, message, null);
+ }
+
+ DataflowJobExecutionException(
+ DataflowPipelineJob job, String message, @Nullable Throwable cause) {
+ super(job, message, cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobUpdatedException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobUpdatedException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobUpdatedException.java
new file mode 100644
index 0000000..e30a0e7
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobUpdatedException.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.dataflow;
+
+/**
+ * Signals that a job run by a {@link BlockingDataflowPipelineRunner} was updated during execution.
+ */
+public class DataflowJobUpdatedException extends DataflowJobException {
+ private DataflowPipelineJob replacedByJob;
+
+ /**
+ * Create a new {@code DataflowJobUpdatedException} with the specified original {@link
+ * DataflowPipelineJob}, message, and replacement {@link DataflowPipelineJob}.
+ */
+ public DataflowJobUpdatedException(
+ DataflowPipelineJob job, String message, DataflowPipelineJob replacedByJob) {
+ this(job, message, replacedByJob, null);
+ }
+
+ /**
+ * Create a new {@code DataflowJobUpdatedException} with the specified original {@link
+ * DataflowPipelineJob}, message, replacement {@link DataflowPipelineJob}, and cause.
+ */
+ public DataflowJobUpdatedException(
+ DataflowPipelineJob job, String message, DataflowPipelineJob replacedByJob, Throwable cause) {
+ super(job, message, cause);
+ this.replacedByJob = replacedByJob;
+ }
+
+ /**
+ * The new job that replaces the job terminated with this exception.
+ */
+ public DataflowPipelineJob getReplacedByJob() {
+ return replacedByJob;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..19df0a1
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
@@ -0,0 +1,397 @@
+/*
+ * 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.dataflow;
+
+import static org.apache.beam.sdk.util.TimeUtil.fromCloudTime;
+
+import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms;
+import org.apache.beam.runners.dataflow.internal.DataflowMetricUpdateExtractor;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.runners.AggregatorRetrievalException;
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.MapAggregatorValues;
+
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.NanoClock;
+import com.google.api.client.util.Sleeper;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.JobMessage;
+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;
+
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.Nullable;
+
+/**
+ * A DataflowPipelineJob represents a job submitted to Dataflow using
+ * {@link DataflowPipelineRunner}.
+ */
+public class DataflowPipelineJob implements PipelineResult {
+ private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineJob.class);
+
+ /**
+ * The id for the job.
+ */
+ private String jobId;
+
+ /**
+ * Google cloud project to associate this pipeline with.
+ */
+ private String projectId;
+
+ /**
+ * Client for the Dataflow service. This can be used to query the service
+ * for information about the job.
+ */
+ private Dataflow dataflowClient;
+
+ /**
+ * The state the job terminated in or {@code null} if the job has not terminated.
+ */
+ @Nullable
+ private State terminalState = null;
+
+ /**
+ * The job that replaced this one or {@code null} if the job has not been replaced.
+ */
+ @Nullable
+ private DataflowPipelineJob replacedByJob = null;
+
+ private DataflowAggregatorTransforms aggregatorTransforms;
+
+ /**
+ * The Metric Updates retrieved after the job was in a terminal state.
+ */
+ private List<MetricUpdate> terminalMetricUpdates;
+
+ /**
+ * The polling interval for job status and messages information.
+ */
+ static final long MESSAGES_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2);
+ static final long STATUS_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2);
+
+ /**
+ * The amount of polling attempts for job status and messages information.
+ */
+ static final int MESSAGES_POLLING_ATTEMPTS = 10;
+ static final int STATUS_POLLING_ATTEMPTS = 5;
+
+ /**
+ * Constructs the job.
+ *
+ * @param projectId the project id
+ * @param jobId the job id
+ * @param dataflowClient the client for the Dataflow Service
+ */
+ public DataflowPipelineJob(String projectId, String jobId, Dataflow dataflowClient,
+ DataflowAggregatorTransforms aggregatorTransforms) {
+ this.projectId = projectId;
+ this.jobId = jobId;
+ this.dataflowClient = dataflowClient;
+ this.aggregatorTransforms = aggregatorTransforms;
+ }
+
+ /**
+ * Get the id of this job.
+ */
+ public String getJobId() {
+ return jobId;
+ }
+
+ /**
+ * Get the project this job exists in.
+ */
+ public String getProjectId() {
+ return projectId;
+ }
+
+ /**
+ * Returns a new {@link DataflowPipelineJob} for the job that replaced this one, if applicable.
+ *
+ * @throws IllegalStateException if called before the job has terminated or if the job terminated
+ * but was not updated
+ */
+ public DataflowPipelineJob getReplacedByJob() {
+ if (terminalState == null) {
+ throw new IllegalStateException("getReplacedByJob() called before job terminated");
+ }
+ if (replacedByJob == null) {
+ throw new IllegalStateException("getReplacedByJob() called for job that was not replaced");
+ }
+ return replacedByJob;
+ }
+
+ /**
+ * Get the Cloud Dataflow API Client used by this job.
+ */
+ public Dataflow getDataflowClient() {
+ return dataflowClient;
+ }
+
+ /**
+ * Waits for the job to finish and return the final status.
+ *
+ * @param timeToWait The time to wait in units timeUnit for the job to finish.
+ * Provide a value less than 1 ms for an infinite wait.
+ * @param timeUnit The unit of time for timeToWait.
+ * @param messageHandler If non null this handler will be invoked for each
+ * batch of messages received.
+ * @return The final state of the job or null on timeout or if the
+ * thread is interrupted.
+ * @throws IOException If there is a persistent problem getting job
+ * information.
+ * @throws InterruptedException
+ */
+ @Nullable
+ public State waitToFinish(
+ long timeToWait,
+ TimeUnit timeUnit,
+ MonitoringUtil.JobMessagesHandler messageHandler)
+ throws IOException, InterruptedException {
+ return waitToFinish(timeToWait, timeUnit, messageHandler, Sleeper.DEFAULT, NanoClock.SYSTEM);
+ }
+
+ /**
+ * Wait for the job to finish and return the final status.
+ *
+ * @param timeToWait The time to wait in units timeUnit for the job to finish.
+ * Provide a value less than 1 ms for an infinite wait.
+ * @param timeUnit The unit of time for timeToWait.
+ * @param messageHandler If non null this handler will be invoked for each
+ * batch of messages received.
+ * @param sleeper A sleeper to use to sleep between attempts.
+ * @param nanoClock A nanoClock used to time the total time taken.
+ * @return The final state of the job or null on timeout or if the
+ * thread is interrupted.
+ * @throws IOException If there is a persistent problem getting job
+ * information.
+ * @throws InterruptedException
+ */
+ @Nullable
+ @VisibleForTesting
+ State waitToFinish(
+ long timeToWait,
+ TimeUnit timeUnit,
+ MonitoringUtil.JobMessagesHandler messageHandler,
+ Sleeper sleeper,
+ NanoClock nanoClock)
+ throws IOException, InterruptedException {
+ MonitoringUtil monitor = new MonitoringUtil(projectId, dataflowClient);
+
+ long lastTimestamp = 0;
+ BackOff backoff =
+ timeUnit.toMillis(timeToWait) > 0
+ ? new AttemptAndTimeBoundedExponentialBackOff(
+ MESSAGES_POLLING_ATTEMPTS,
+ MESSAGES_POLLING_INTERVAL,
+ timeUnit.toMillis(timeToWait),
+ AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS,
+ nanoClock)
+ : new AttemptBoundedExponentialBackOff(
+ MESSAGES_POLLING_ATTEMPTS, MESSAGES_POLLING_INTERVAL);
+ State state;
+ do {
+ // Get the state of the job before listing messages. This ensures we always fetch job
+ // messages after the job finishes to ensure we have all them.
+ state = getStateWithRetries(1, sleeper);
+ boolean hasError = state == State.UNKNOWN;
+
+ if (messageHandler != null && !hasError) {
+ // Process all the job messages that have accumulated so far.
+ try {
+ List<JobMessage> allMessages = monitor.getJobMessages(
+ jobId, lastTimestamp);
+
+ if (!allMessages.isEmpty()) {
+ lastTimestamp =
+ fromCloudTime(allMessages.get(allMessages.size() - 1).getTime()).getMillis();
+ messageHandler.process(allMessages);
+ }
+ } catch (GoogleJsonResponseException | SocketTimeoutException e) {
+ hasError = true;
+ LOG.warn("There were problems getting current job messages: {}.", e.getMessage());
+ LOG.debug("Exception information:", e);
+ }
+ }
+
+ if (!hasError) {
+ backoff.reset();
+ // Check if the job is done.
+ if (state.isTerminal()) {
+ return state;
+ }
+ }
+ } while(BackOffUtils.next(sleeper, backoff));
+ LOG.warn("No terminal state was returned. State value {}", state);
+ return null; // Timed out.
+ }
+
+ /**
+ * Cancels the job.
+ * @throws IOException if there is a problem executing the cancel request.
+ */
+ public void cancel() throws IOException {
+ Job content = new Job();
+ content.setProjectId(projectId);
+ content.setId(jobId);
+ content.setRequestedState("JOB_STATE_CANCELLED");
+ dataflowClient.projects().jobs()
+ .update(projectId, jobId, content)
+ .execute();
+ }
+
+ @Override
+ public State getState() {
+ if (terminalState != null) {
+ return terminalState;
+ }
+
+ return getStateWithRetries(STATUS_POLLING_ATTEMPTS, Sleeper.DEFAULT);
+ }
+
+ /**
+ * Attempts to get the state. Uses exponential backoff on failure up to the maximum number
+ * of passed in attempts.
+ *
+ * @param attempts The amount of attempts to make.
+ * @param sleeper Object used to do the sleeps between attempts.
+ * @return The state of the job or State.UNKNOWN in case of failure.
+ */
+ @VisibleForTesting
+ State getStateWithRetries(int attempts, Sleeper sleeper) {
+ if (terminalState != null) {
+ return terminalState;
+ }
+ try {
+ Job job = getJobWithRetries(attempts, sleeper);
+ return MonitoringUtil.toState(job.getCurrentState());
+ } catch (IOException exn) {
+ // The only IOException that getJobWithRetries is permitted to throw is the final IOException
+ // that caused the failure of retry. Other exceptions are wrapped in an unchecked exceptions
+ // and will propagate.
+ return State.UNKNOWN;
+ }
+ }
+
+ /**
+ * Attempts to get the underlying {@link Job}. Uses exponential backoff on failure up to the
+ * maximum number of passed in attempts.
+ *
+ * @param attempts The amount of attempts to make.
+ * @param sleeper Object used to do the sleeps between attempts.
+ * @return The underlying {@link Job} object.
+ * @throws IOException When the maximum number of retries is exhausted, the last exception is
+ * thrown.
+ */
+ @VisibleForTesting
+ Job getJobWithRetries(int attempts, Sleeper sleeper) throws IOException {
+ AttemptBoundedExponentialBackOff backoff =
+ new AttemptBoundedExponentialBackOff(attempts, STATUS_POLLING_INTERVAL);
+
+ // Retry loop ends in return or throw
+ while (true) {
+ try {
+ Job job = dataflowClient
+ .projects()
+ .jobs()
+ .get(projectId, jobId)
+ .execute();
+ State currentState = MonitoringUtil.toState(job.getCurrentState());
+ if (currentState.isTerminal()) {
+ terminalState = currentState;
+ replacedByJob = new DataflowPipelineJob(
+ getProjectId(), job.getReplacedByJobId(), dataflowClient, aggregatorTransforms);
+ }
+ return job;
+ } catch (IOException exn) {
+ LOG.warn("There were problems getting current job status: {}.", exn.getMessage());
+ LOG.debug("Exception information:", exn);
+
+ if (!nextBackOff(sleeper, backoff)) {
+ throw exn;
+ }
+ }
+ }
+ }
+
+ /**
+ * Identical to {@link BackOffUtils#next} but without checked exceptions.
+ */
+ private boolean nextBackOff(Sleeper sleeper, BackOff backoff) {
+ try {
+ return BackOffUtils.next(sleeper, backoff);
+ } catch (InterruptedException | IOException e) {
+ if (e instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
+ throw Throwables.propagate(e);
+ }
+ }
+
+ @Override
+ public <OutputT> AggregatorValues<OutputT> getAggregatorValues(Aggregator<?, OutputT> aggregator)
+ throws AggregatorRetrievalException {
+ try {
+ return new MapAggregatorValues<>(fromMetricUpdates(aggregator));
+ } catch (IOException e) {
+ throw new AggregatorRetrievalException(
+ "IOException when retrieving Aggregator values for Aggregator " + aggregator, e);
+ }
+ }
+
+ private <OutputT> Map<String, OutputT> fromMetricUpdates(Aggregator<?, OutputT> aggregator)
+ throws IOException {
+ if (aggregatorTransforms.contains(aggregator)) {
+ List<MetricUpdate> metricUpdates;
+ if (terminalMetricUpdates != null) {
+ metricUpdates = terminalMetricUpdates;
+ } else {
+ boolean terminal = getState().isTerminal();
+ JobMetrics jobMetrics =
+ dataflowClient.projects().jobs().getMetrics(projectId, jobId).execute();
+ metricUpdates = jobMetrics.getMetrics();
+ if (terminal && jobMetrics.getMetrics() != null) {
+ terminalMetricUpdates = metricUpdates;
+ }
+ }
+
+ return DataflowMetricUpdateExtractor.fromMetricUpdates(
+ aggregator, aggregatorTransforms, metricUpdates);
+ } else {
+ throw new IllegalArgumentException(
+ "Aggregator " + aggregator + " is not used in this pipeline");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
new file mode 100644
index 0000000..d7d243f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java
@@ -0,0 +1,62 @@
+/*
+ * 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.dataflow;
+
+import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
+
+import com.google.auto.service.AutoService;
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for the
+ * {@link DataflowPipelineRunner}.
+ */
+public class DataflowPipelineRegistrar {
+ private DataflowPipelineRegistrar() { }
+
+ /**
+ * Register the {@link DataflowPipelineOptions} and {@link BlockingDataflowPipelineOptions}.
+ */
+ @AutoService(PipelineOptionsRegistrar.class)
+ public static class Options implements PipelineOptionsRegistrar {
+ @Override
+ public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
+ return ImmutableList.<Class<? extends PipelineOptions>>of(
+ DataflowPipelineOptions.class,
+ BlockingDataflowPipelineOptions.class);
+ }
+ }
+
+ /**
+ * Register the {@link DataflowPipelineRunner} and {@link BlockingDataflowPipelineRunner}.
+ */
+ @AutoService(PipelineRunnerRegistrar.class)
+ public static class Runner implements PipelineRunnerRegistrar {
+ @Override
+ public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
+ return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
+ DataflowPipelineRunner.class,
+ BlockingDataflowPipelineRunner.class);
+ }
+ }
+}
[10/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java
deleted file mode 100644
index 7ea44d7..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunnerHooks.java
+++ /dev/null
@@ -1,39 +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.runners;
-
-import org.apache.beam.sdk.annotations.Experimental;
-
-import com.google.api.services.dataflow.model.Environment;
-
-/**
- * An instance of this class can be passed to the
- * {@link DataflowPipelineRunner} to add user defined hooks to be
- * invoked at various times during pipeline execution.
- */
-@Experimental
-public class DataflowPipelineRunnerHooks {
- /**
- * Allows the user to modify the environment of their job before their job is submitted
- * to the service for execution.
- *
- * @param environment The environment of the job. Users can make change to this instance in order
- * to change the environment with which their job executes on the service.
- */
- public void modifyEnvironmentBeforeSubmission(Environment environment) {}
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java
deleted file mode 100644
index 5c0745f..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineTranslator.java
+++ /dev/null
@@ -1,1058 +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.runners;
-
-import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray;
-import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString;
-import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray;
-import static org.apache.beam.sdk.util.Structs.addBoolean;
-import static org.apache.beam.sdk.util.Structs.addDictionary;
-import static org.apache.beam.sdk.util.Structs.addList;
-import static org.apache.beam.sdk.util.Structs.addLong;
-import static org.apache.beam.sdk.util.Structs.addObject;
-import static org.apache.beam.sdk.util.Structs.addString;
-import static org.apache.beam.sdk.util.Structs.getString;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.PubsubIO;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.StreamingOptions;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner.GroupByKeyAndSortValuesOnly;
-import org.apache.beam.sdk.runners.dataflow.BigQueryIOTranslator;
-import org.apache.beam.sdk.runners.dataflow.PubsubIOTranslator;
-import org.apache.beam.sdk.runners.dataflow.ReadTranslator;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Combine;
-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.display.DisplayData;
-import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.AppliedCombineFn;
-import org.apache.beam.sdk.util.CloudObject;
-import org.apache.beam.sdk.util.DoFnInfo;
-import org.apache.beam.sdk.util.OutputReference;
-import org.apache.beam.sdk.util.PropertyNames;
-import org.apache.beam.sdk.util.SerializableUtils;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-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.TupleTag;
-import org.apache.beam.sdk.values.TypedPValue;
-
-import com.google.api.services.dataflow.model.AutoscalingSettings;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.api.services.dataflow.model.Disk;
-import com.google.api.services.dataflow.model.Environment;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.Step;
-import com.google.api.services.dataflow.model.WorkerPool;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import javax.annotation.Nullable;
-
-/**
- * {@link DataflowPipelineTranslator} knows how to translate {@link Pipeline} objects
- * into Cloud Dataflow Service API {@link Job}s.
- */
-@SuppressWarnings({"rawtypes", "unchecked"})
-public class DataflowPipelineTranslator {
- // Must be kept in sync with their internal counterparts.
- private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineTranslator.class);
- private static final ObjectMapper MAPPER = new ObjectMapper();
-
- /**
- * A map from {@link PTransform} subclass to the corresponding
- * {@link TransformTranslator} to use to translate that transform.
- *
- * <p>A static map that contains system-wide defaults.
- */
- private static Map<Class, TransformTranslator> transformTranslators =
- new HashMap<>();
-
- /** Provided configuration options. */
- private final DataflowPipelineOptions options;
-
- /**
- * Constructs a translator from the provided options.
- *
- * @param options Properties that configure the translator.
- *
- * @return The newly created translator.
- */
- public static DataflowPipelineTranslator fromOptions(
- DataflowPipelineOptions options) {
- return new DataflowPipelineTranslator(options);
- }
-
- private DataflowPipelineTranslator(DataflowPipelineOptions options) {
- this.options = options;
- }
-
- /**
- * Translates a {@link Pipeline} into a {@code JobSpecification}.
- */
- public JobSpecification translate(
- Pipeline pipeline,
- DataflowPipelineRunner runner,
- List<DataflowPackage> packages) {
-
- Translator translator = new Translator(pipeline, runner);
- Job result = translator.translate(packages);
- return new JobSpecification(result, Collections.unmodifiableMap(translator.stepNames));
- }
-
- /**
- * The result of a job translation.
- *
- * <p>Used to pass the result {@link Job} and any state that was used to construct the job that
- * may be of use to other classes (eg the {@link PTransform} to StepName mapping).
- */
- public static class JobSpecification {
- private final Job job;
- private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
-
- public JobSpecification(Job job, Map<AppliedPTransform<?, ?, ?>, String> stepNames) {
- this.job = job;
- this.stepNames = stepNames;
- }
-
- public Job getJob() {
- return job;
- }
-
- /**
- * Returns the mapping of {@link AppliedPTransform AppliedPTransforms} to the internal step
- * name for that {@code AppliedPTransform}.
- */
- public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
- return stepNames;
- }
- }
-
- /**
- * Renders a {@link Job} as a string.
- */
- public static String jobToString(Job job) {
- try {
- return MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(job);
- } catch (JsonProcessingException exc) {
- throw new IllegalStateException("Failed to render Job as String.", exc);
- }
- }
-
- /////////////////////////////////////////////////////////////////////////////
-
- /**
- * Records that instances of the specified PTransform class
- * should be translated by default by the corresponding
- * {@link TransformTranslator}.
- */
- public static <TransformT extends PTransform> void registerTransformTranslator(
- Class<TransformT> transformClass,
- TransformTranslator<? extends TransformT> transformTranslator) {
- if (transformTranslators.put(transformClass, transformTranslator) != null) {
- throw new IllegalArgumentException(
- "defining multiple translators for " + transformClass);
- }
- }
-
- /**
- * Returns the {@link TransformTranslator} to use for instances of the
- * specified PTransform class, or null if none registered.
- */
- public <TransformT extends PTransform>
- TransformTranslator<TransformT> getTransformTranslator(Class<TransformT> transformClass) {
- return transformTranslators.get(transformClass);
- }
-
- /**
- * A {@link TransformTranslator} knows how to translate
- * a particular subclass of {@link PTransform} for the
- * Cloud Dataflow service. It does so by
- * mutating the {@link TranslationContext}.
- */
- public interface TransformTranslator<TransformT extends PTransform> {
- public void translate(TransformT transform,
- TranslationContext context);
- }
-
- /**
- * The interface provided to registered callbacks for interacting
- * with the {@link DataflowPipelineRunner}, including reading and writing the
- * values of {@link PCollection}s and side inputs ({@link PCollectionView}s).
- */
- public interface TranslationContext {
- /**
- * Returns the configured pipeline options.
- */
- DataflowPipelineOptions getPipelineOptions();
-
- /**
- * Returns the input of the currently being translated transform.
- */
- <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform);
-
- /**
- * Returns the output of the currently being translated transform.
- */
- <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform);
-
- /**
- * Returns the full name of the currently being translated transform.
- */
- String getFullName(PTransform<?, ?> transform);
-
- /**
- * Adds a step to the Dataflow workflow for the given transform, with
- * the given Dataflow step type.
- * This step becomes "current" for the purpose of {@link #addInput} and
- * {@link #addOutput}.
- */
- public void addStep(PTransform<?, ?> transform, String type);
-
- /**
- * Adds a pre-defined step to the Dataflow workflow. The given PTransform should be
- * consistent with the Step, in terms of input, output and coder types.
- *
- * <p>This is a low-level operation, when using this method it is up to
- * the caller to ensure that names do not collide.
- */
- public void addStep(PTransform<?, ? extends PValue> transform, Step step);
-
- /**
- * Sets the encoding for the current Dataflow step.
- */
- public void addEncodingInput(Coder<?> value);
-
- /**
- * Adds an input with the given name and value to the current
- * Dataflow step.
- */
- public void addInput(String name, Boolean value);
-
- /**
- * Adds an input with the given name and value to the current
- * Dataflow step.
- */
- public void addInput(String name, String value);
-
- /**
- * Adds an input with the given name and value to the current
- * Dataflow step.
- */
- public void addInput(String name, Long value);
-
- /**
- * Adds an input with the given name to the previously added Dataflow
- * step, coming from the specified input PValue.
- */
- public void addInput(String name, PInput value);
-
- /**
- * Adds an input that is a dictionary of strings to objects.
- */
- public void addInput(String name, Map<String, Object> elements);
-
- /**
- * Adds an input that is a list of objects.
- */
- public void addInput(String name, List<? extends Map<String, Object>> elements);
-
- /**
- * Adds an output with the given name to the previously added
- * Dataflow step, producing the specified output {@code PValue},
- * including its {@code Coder} if a {@code TypedPValue}. If the
- * {@code PValue} is a {@code PCollection}, wraps its coder inside
- * a {@code WindowedValueCoder}.
- */
- public void addOutput(String name, PValue value);
-
- /**
- * Adds an output with the given name to the previously added
- * Dataflow step, producing the specified output {@code PValue},
- * including its {@code Coder} if a {@code TypedPValue}. If the
- * {@code PValue} is a {@code PCollection}, wraps its coder inside
- * a {@code ValueOnlyCoder}.
- */
- public void addValueOnlyOutput(String name, PValue value);
-
- /**
- * Adds an output with the given name to the previously added
- * CollectionToSingleton Dataflow step, consuming the specified
- * input {@code PValue} and producing the specified output
- * {@code PValue}. This step requires special treatment for its
- * output encoding.
- */
- public void addCollectionToSingletonOutput(String name,
- PValue inputValue,
- PValue outputValue);
-
- /**
- * Encode a PValue reference as an output reference.
- */
- public OutputReference asOutputReference(PValue value);
- }
-
-
- /////////////////////////////////////////////////////////////////////////////
-
- /**
- * Translates a Pipeline into the Dataflow representation.
- */
- class Translator implements PipelineVisitor, TranslationContext {
- /** The Pipeline to translate. */
- private final Pipeline pipeline;
-
- /** The runner which will execute the pipeline. */
- private final DataflowPipelineRunner runner;
-
- /** The Cloud Dataflow Job representation. */
- private final Job job = new Job();
-
- /**
- * Translator is stateful, as addProperty calls refer to the current step.
- */
- private Step currentStep;
-
- /**
- * A Map from AppliedPTransform to their unique Dataflow step names.
- */
- private final Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
-
- /**
- * A Map from PValues to their output names used by their producer
- * Dataflow steps.
- */
- private final Map<POutput, String> outputNames = new HashMap<>();
-
- /**
- * A Map from PValues to the Coders used for them.
- */
- private final Map<POutput, Coder<?>> outputCoders = new HashMap<>();
-
- /**
- * The transform currently being applied.
- */
- private AppliedPTransform<?, ?, ?> currentTransform;
-
- /**
- * Constructs a Translator that will translate the specified
- * Pipeline into Dataflow objects.
- */
- public Translator(Pipeline pipeline, DataflowPipelineRunner runner) {
- this.pipeline = pipeline;
- this.runner = runner;
- }
-
- /**
- * Translates this Translator's pipeline onto its writer.
- * @return a Job definition filled in with the type of job, the environment,
- * and the job steps.
- */
- public Job translate(List<DataflowPackage> packages) {
- job.setName(options.getJobName().toLowerCase());
-
- Environment environment = new Environment();
- job.setEnvironment(environment);
-
- try {
- environment.setSdkPipelineOptions(
- MAPPER.readValue(MAPPER.writeValueAsBytes(options), Map.class));
- } catch (IOException e) {
- throw new IllegalArgumentException(
- "PipelineOptions specified failed to serialize to JSON.", e);
- }
-
- WorkerPool workerPool = new WorkerPool();
-
- if (options.getTeardownPolicy() != null) {
- workerPool.setTeardownPolicy(options.getTeardownPolicy().getTeardownPolicyName());
- }
-
- if (options.isStreaming()) {
- job.setType("JOB_TYPE_STREAMING");
- } else {
- job.setType("JOB_TYPE_BATCH");
- workerPool.setDiskType(options.getWorkerDiskType());
- }
-
- if (options.getWorkerMachineType() != null) {
- workerPool.setMachineType(options.getWorkerMachineType());
- }
-
- workerPool.setPackages(packages);
- workerPool.setNumWorkers(options.getNumWorkers());
-
- if (options.isStreaming()) {
- // Use separate data disk for streaming.
- Disk disk = new Disk();
- disk.setDiskType(options.getWorkerDiskType());
- workerPool.setDataDisks(Collections.singletonList(disk));
- }
- if (!Strings.isNullOrEmpty(options.getZone())) {
- workerPool.setZone(options.getZone());
- }
- if (!Strings.isNullOrEmpty(options.getNetwork())) {
- workerPool.setNetwork(options.getNetwork());
- }
- if (!Strings.isNullOrEmpty(options.getSubnetwork())) {
- workerPool.setSubnetwork(options.getSubnetwork());
- }
- if (options.getDiskSizeGb() > 0) {
- workerPool.setDiskSizeGb(options.getDiskSizeGb());
- }
- AutoscalingSettings settings = new AutoscalingSettings();
- if (options.getAutoscalingAlgorithm() != null) {
- settings.setAlgorithm(options.getAutoscalingAlgorithm().getAlgorithm());
- }
- settings.setMaxNumWorkers(options.getMaxNumWorkers());
- workerPool.setAutoscalingSettings(settings);
-
- List<WorkerPool> workerPools = new LinkedList<>();
-
- workerPools.add(workerPool);
- environment.setWorkerPools(workerPools);
-
- pipeline.traverseTopologically(this);
- return job;
- }
-
- @Override
- public DataflowPipelineOptions getPipelineOptions() {
- return options;
- }
-
- @Override
- public <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform) {
- return (InputT) getCurrentTransform(transform).getInput();
- }
-
- @Override
- public <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform) {
- return (OutputT) getCurrentTransform(transform).getOutput();
- }
-
- @Override
- public String getFullName(PTransform<?, ?> transform) {
- return getCurrentTransform(transform).getFullName();
- }
-
- private AppliedPTransform<?, ?, ?> getCurrentTransform(PTransform<?, ?> transform) {
- checkArgument(
- currentTransform != null && currentTransform.getTransform() == transform,
- "can only be called with current transform");
- return currentTransform;
- }
-
- @Override
- public void enterCompositeTransform(TransformTreeNode node) {
- }
-
- @Override
- public void leaveCompositeTransform(TransformTreeNode node) {
- }
-
- @Override
- public void visitTransform(TransformTreeNode node) {
- PTransform<?, ?> transform = node.getTransform();
- TransformTranslator translator =
- getTransformTranslator(transform.getClass());
- if (translator == null) {
- throw new IllegalStateException(
- "no translator registered for " + transform);
- }
- LOG.debug("Translating {}", transform);
- currentTransform = AppliedPTransform.of(
- node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform);
- translator.translate(transform, this);
- currentTransform = null;
- }
-
- @Override
- public void visitValue(PValue value, TransformTreeNode producer) {
- LOG.debug("Checking translation of {}", value);
- if (value.getProducingTransformInternal() == null) {
- throw new RuntimeException(
- "internal error: expecting a PValue "
- + "to have a producingTransform");
- }
- if (!producer.isCompositeNode()) {
- // Primitive transforms are the only ones assigned step names.
- asOutputReference(value);
- }
- }
-
- @Override
- public void addStep(PTransform<?, ?> transform, String type) {
- String stepName = genStepName();
- if (stepNames.put(getCurrentTransform(transform), stepName) != null) {
- throw new IllegalArgumentException(
- transform + " already has a name specified");
- }
- // Start the next "steps" list item.
- List<Step> steps = job.getSteps();
- if (steps == null) {
- steps = new LinkedList<>();
- job.setSteps(steps);
- }
-
- currentStep = new Step();
- currentStep.setName(stepName);
- currentStep.setKind(type);
- steps.add(currentStep);
- addInput(PropertyNames.USER_NAME, getFullName(transform));
- addDisplayData(PropertyNames.DISPLAY_DATA, DisplayData.from(transform));
- }
-
- @Override
- public void addStep(PTransform<?, ? extends PValue> transform, Step original) {
- Step step = original.clone();
- String stepName = step.getName();
- if (stepNames.put(getCurrentTransform(transform), stepName) != null) {
- throw new IllegalArgumentException(transform + " already has a name specified");
- }
-
- Map<String, Object> properties = step.getProperties();
- if (properties != null) {
- @Nullable List<Map<String, Object>> outputInfoList = null;
- try {
- // TODO: This should be done via a Structs accessor.
- @Nullable List<Map<String, Object>> list =
- (List<Map<String, Object>>) properties.get(PropertyNames.OUTPUT_INFO);
- outputInfoList = list;
- } catch (Exception e) {
- throw new RuntimeException("Inconsistent dataflow pipeline translation", e);
- }
- if (outputInfoList != null && outputInfoList.size() > 0) {
- Map<String, Object> firstOutputPort = outputInfoList.get(0);
- @Nullable String name;
- try {
- name = getString(firstOutputPort, PropertyNames.OUTPUT_NAME);
- } catch (Exception e) {
- name = null;
- }
- if (name != null) {
- registerOutputName(getOutput(transform), name);
- }
- }
- }
-
- List<Step> steps = job.getSteps();
- if (steps == null) {
- steps = new LinkedList<>();
- job.setSteps(steps);
- }
- currentStep = step;
- steps.add(step);
- }
-
- @Override
- public void addEncodingInput(Coder<?> coder) {
- CloudObject encoding = SerializableUtils.ensureSerializable(coder);
- addObject(getProperties(), PropertyNames.ENCODING, encoding);
- }
-
- @Override
- public void addInput(String name, Boolean value) {
- addBoolean(getProperties(), name, value);
- }
-
- @Override
- public void addInput(String name, String value) {
- addString(getProperties(), name, value);
- }
-
- @Override
- public void addInput(String name, Long value) {
- addLong(getProperties(), name, value);
- }
-
- @Override
- public void addInput(String name, Map<String, Object> elements) {
- addDictionary(getProperties(), name, elements);
- }
-
- @Override
- public void addInput(String name, List<? extends Map<String, Object>> elements) {
- addList(getProperties(), name, elements);
- }
-
- @Override
- public void addInput(String name, PInput value) {
- if (value instanceof PValue) {
- addInput(name, asOutputReference((PValue) value));
- } else {
- throw new IllegalStateException("Input must be a PValue");
- }
- }
-
- @Override
- public void addOutput(String name, PValue value) {
- Coder<?> coder;
- if (value instanceof TypedPValue) {
- coder = ((TypedPValue<?>) value).getCoder();
- if (value instanceof PCollection) {
- // Wrap the PCollection element Coder inside a WindowedValueCoder.
- coder = WindowedValue.getFullCoder(
- coder,
- ((PCollection<?>) value).getWindowingStrategy().getWindowFn().windowCoder());
- }
- } else {
- // No output coder to encode.
- coder = null;
- }
- addOutput(name, value, coder);
- }
-
- @Override
- public void addValueOnlyOutput(String name, PValue value) {
- Coder<?> coder;
- if (value instanceof TypedPValue) {
- coder = ((TypedPValue<?>) value).getCoder();
- if (value instanceof PCollection) {
- // Wrap the PCollection element Coder inside a ValueOnly
- // WindowedValueCoder.
- coder = WindowedValue.getValueOnlyCoder(coder);
- }
- } else {
- // No output coder to encode.
- coder = null;
- }
- addOutput(name, value, coder);
- }
-
- @Override
- public void addCollectionToSingletonOutput(String name,
- PValue inputValue,
- PValue outputValue) {
- Coder<?> inputValueCoder =
- Preconditions.checkNotNull(outputCoders.get(inputValue));
- // The inputValueCoder for the input PCollection should be some
- // WindowedValueCoder of the input PCollection's element
- // coder.
- Preconditions.checkState(
- inputValueCoder instanceof WindowedValue.WindowedValueCoder);
- // The outputValueCoder for the output should be an
- // IterableCoder of the inputValueCoder. This is a property
- // of the backend "CollectionToSingleton" step.
- Coder<?> outputValueCoder = IterableCoder.of(inputValueCoder);
- addOutput(name, outputValue, outputValueCoder);
- }
-
- /**
- * Adds an output with the given name to the previously added
- * Dataflow step, producing the specified output {@code PValue}
- * with the given {@code Coder} (if not {@code null}).
- */
- private void addOutput(String name, PValue value, Coder<?> valueCoder) {
- registerOutputName(value, name);
-
- Map<String, Object> properties = getProperties();
- @Nullable List<Map<String, Object>> outputInfoList = null;
- try {
- // TODO: This should be done via a Structs accessor.
- outputInfoList = (List<Map<String, Object>>) properties.get(PropertyNames.OUTPUT_INFO);
- } catch (Exception e) {
- throw new RuntimeException("Inconsistent dataflow pipeline translation", e);
- }
- if (outputInfoList == null) {
- outputInfoList = new ArrayList<>();
- // TODO: This should be done via a Structs accessor.
- properties.put(PropertyNames.OUTPUT_INFO, outputInfoList);
- }
-
- Map<String, Object> outputInfo = new HashMap<>();
- addString(outputInfo, PropertyNames.OUTPUT_NAME, name);
- addString(outputInfo, PropertyNames.USER_NAME, value.getName());
- if (value instanceof PCollection
- && runner.doesPCollectionRequireIndexedFormat((PCollection<?>) value)) {
- addBoolean(outputInfo, PropertyNames.USE_INDEXED_FORMAT, true);
- }
- if (valueCoder != null) {
- // Verify that encoding can be decoded, in order to catch serialization
- // failures as early as possible.
- CloudObject encoding = SerializableUtils.ensureSerializable(valueCoder);
- addObject(outputInfo, PropertyNames.ENCODING, encoding);
- outputCoders.put(value, valueCoder);
- }
-
- outputInfoList.add(outputInfo);
- }
-
- private void addDisplayData(String name, DisplayData displayData) {
- List<Map<String, Object>> list = MAPPER.convertValue(displayData, List.class);
- addList(getProperties(), name, list);
- }
-
- @Override
- public OutputReference asOutputReference(PValue value) {
- AppliedPTransform<?, ?, ?> transform =
- value.getProducingTransformInternal();
- String stepName = stepNames.get(transform);
- if (stepName == null) {
- throw new IllegalArgumentException(transform + " doesn't have a name specified");
- }
-
- String outputName = outputNames.get(value);
- if (outputName == null) {
- throw new IllegalArgumentException(
- "output " + value + " doesn't have a name specified");
- }
-
- return new OutputReference(stepName, outputName);
- }
-
- private Map<String, Object> getProperties() {
- Map<String, Object> properties = currentStep.getProperties();
- if (properties == null) {
- properties = new HashMap<>();
- currentStep.setProperties(properties);
- }
- return properties;
- }
-
- /**
- * Returns a fresh Dataflow step name.
- */
- private String genStepName() {
- return "s" + (stepNames.size() + 1);
- }
-
- /**
- * Records the name of the given output PValue,
- * within its producing transform.
- */
- private void registerOutputName(POutput value, String name) {
- if (outputNames.put(value, name) != null) {
- throw new IllegalArgumentException(
- "output " + value + " already has a name specified");
- }
- }
- }
-
- /////////////////////////////////////////////////////////////////////////////
-
- @Override
- public String toString() {
- return "DataflowPipelineTranslator#" + hashCode();
- }
-
-
- ///////////////////////////////////////////////////////////////////////////
-
- static {
- registerTransformTranslator(
- View.CreatePCollectionView.class,
- new TransformTranslator<View.CreatePCollectionView>() {
- @Override
- public void translate(
- View.CreatePCollectionView transform,
- TranslationContext context) {
- translateTyped(transform, context);
- }
-
- private <ElemT, ViewT> void translateTyped(
- View.CreatePCollectionView<ElemT, ViewT> transform,
- TranslationContext context) {
- context.addStep(transform, "CollectionToSingleton");
- context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
- context.addCollectionToSingletonOutput(
- PropertyNames.OUTPUT,
- context.getInput(transform),
- context.getOutput(transform));
- }
- });
-
- DataflowPipelineTranslator.registerTransformTranslator(
- Combine.GroupedValues.class,
- new DataflowPipelineTranslator.TransformTranslator<Combine.GroupedValues>() {
- @Override
- public void translate(
- Combine.GroupedValues transform,
- DataflowPipelineTranslator.TranslationContext context) {
- translateHelper(transform, context);
- }
-
- private <K, InputT, OutputT> void translateHelper(
- final Combine.GroupedValues<K, InputT, OutputT> transform,
- DataflowPipelineTranslator.TranslationContext context) {
- context.addStep(transform, "CombineValues");
- translateInputs(context.getInput(transform), transform.getSideInputs(), context);
-
- AppliedCombineFn<? super K, ? super InputT, ?, OutputT> fn =
- transform.getAppliedFn(
- context.getInput(transform).getPipeline().getCoderRegistry(),
- context.getInput(transform).getCoder(),
- context.getInput(transform).getWindowingStrategy());
-
- context.addEncodingInput(fn.getAccumulatorCoder());
- context.addInput(
- PropertyNames.SERIALIZED_FN,
- byteArrayToJsonString(serializeToByteArray(fn)));
- context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
- }
- });
-
- registerTransformTranslator(
- Flatten.FlattenPCollectionList.class,
- new TransformTranslator<Flatten.FlattenPCollectionList>() {
- @Override
- public void translate(
- Flatten.FlattenPCollectionList transform,
- TranslationContext context) {
- flattenHelper(transform, context);
- }
-
- private <T> void flattenHelper(
- Flatten.FlattenPCollectionList<T> transform,
- TranslationContext context) {
- context.addStep(transform, "Flatten");
-
- List<OutputReference> inputs = new LinkedList<>();
- for (PCollection<T> input : context.getInput(transform).getAll()) {
- inputs.add(context.asOutputReference(input));
- }
- context.addInput(PropertyNames.INPUTS, inputs);
- context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
- }
- });
-
- registerTransformTranslator(
- GroupByKeyAndSortValuesOnly.class,
- new TransformTranslator<GroupByKeyAndSortValuesOnly>() {
- @Override
- public void translate(
- GroupByKeyAndSortValuesOnly transform,
- TranslationContext context) {
- groupByKeyAndSortValuesHelper(transform, context);
- }
-
- private <K1, K2, V> void groupByKeyAndSortValuesHelper(
- GroupByKeyAndSortValuesOnly<K1, K2, V> transform,
- TranslationContext context) {
- context.addStep(transform, "GroupByKey");
- context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
- context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
- context.addInput(PropertyNames.SORT_VALUES, true);
-
- // TODO: Add support for combiner lifting once the need arises.
- context.addInput(
- PropertyNames.DISALLOW_COMBINER_LIFTING, true);
- }
- });
-
- registerTransformTranslator(
- GroupByKey.class,
- new TransformTranslator<GroupByKey>() {
- @Override
- public void translate(
- GroupByKey transform,
- TranslationContext context) {
- groupByKeyHelper(transform, context);
- }
-
- private <K, V> void groupByKeyHelper(
- GroupByKey<K, V> transform,
- TranslationContext context) {
- context.addStep(transform, "GroupByKey");
- context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
- context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-
- WindowingStrategy<?, ?> windowingStrategy =
- context.getInput(transform).getWindowingStrategy();
- boolean isStreaming =
- context.getPipelineOptions().as(StreamingOptions.class).isStreaming();
- boolean disallowCombinerLifting =
- !windowingStrategy.getWindowFn().isNonMerging()
- || (isStreaming && !transform.fewKeys())
- // TODO: Allow combiner lifting on the non-default trigger, as appropriate.
- || !(windowingStrategy.getTrigger().getSpec() instanceof DefaultTrigger);
- context.addInput(
- PropertyNames.DISALLOW_COMBINER_LIFTING, disallowCombinerLifting);
- context.addInput(
- PropertyNames.SERIALIZED_FN,
- byteArrayToJsonString(serializeToByteArray(windowingStrategy)));
- context.addInput(
- PropertyNames.IS_MERGING_WINDOW_FN,
- !windowingStrategy.getWindowFn().isNonMerging());
- }
- });
-
- registerTransformTranslator(
- ParDo.BoundMulti.class,
- new TransformTranslator<ParDo.BoundMulti>() {
- @Override
- public void translate(
- ParDo.BoundMulti transform,
- TranslationContext context) {
- translateMultiHelper(transform, context);
- }
-
- private <InputT, OutputT> void translateMultiHelper(
- ParDo.BoundMulti<InputT, OutputT> transform,
- TranslationContext context) {
- context.addStep(transform, "ParallelDo");
- translateInputs(context.getInput(transform), transform.getSideInputs(), context);
- translateFn(transform.getFn(), context.getInput(transform).getWindowingStrategy(),
- transform.getSideInputs(), context.getInput(transform).getCoder(), context);
- translateOutputs(context.getOutput(transform), context);
- }
- });
-
- registerTransformTranslator(
- ParDo.Bound.class,
- new TransformTranslator<ParDo.Bound>() {
- @Override
- public void translate(
- ParDo.Bound transform,
- TranslationContext context) {
- translateSingleHelper(transform, context);
- }
-
- private <InputT, OutputT> void translateSingleHelper(
- ParDo.Bound<InputT, OutputT> transform,
- TranslationContext context) {
- context.addStep(transform, "ParallelDo");
- translateInputs(context.getInput(transform), transform.getSideInputs(), context);
- translateFn(
- transform.getFn(),
- context.getInput(transform).getWindowingStrategy(),
- transform.getSideInputs(), context.getInput(transform).getCoder(), context);
- context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
- }
- });
-
- registerTransformTranslator(
- Window.Bound.class,
- new DataflowPipelineTranslator.TransformTranslator<Window.Bound>() {
- @Override
- public void translate(
- Window.Bound transform, TranslationContext context) {
- translateHelper(transform, context);
- }
-
- private <T> void translateHelper(
- Window.Bound<T> transform, TranslationContext context) {
- context.addStep(transform, "Bucket");
- context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
- context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
-
- WindowingStrategy<?, ?> strategy = context.getOutput(transform).getWindowingStrategy();
- byte[] serializedBytes = serializeToByteArray(strategy);
- String serializedJson = byteArrayToJsonString(serializedBytes);
- assert Arrays.equals(serializedBytes,
- jsonStringToByteArray(serializedJson));
- context.addInput(PropertyNames.SERIALIZED_FN, serializedJson);
- }
- });
-
- ///////////////////////////////////////////////////////////////////////////
- // IO Translation.
-
- registerTransformTranslator(
- BigQueryIO.Read.Bound.class, new BigQueryIOTranslator.ReadTranslator());
-
- registerTransformTranslator(
- PubsubIO.Read.Bound.class, new PubsubIOTranslator.ReadTranslator());
- registerTransformTranslator(
- DataflowPipelineRunner.StreamingPubsubIOWrite.class,
- new PubsubIOTranslator.WriteTranslator());
-
- registerTransformTranslator(Read.Bounded.class, new ReadTranslator());
- }
-
- private static void translateInputs(
- PCollection<?> input,
- List<PCollectionView<?>> sideInputs,
- TranslationContext context) {
- context.addInput(PropertyNames.PARALLEL_INPUT, input);
- translateSideInputs(sideInputs, context);
- }
-
- // Used for ParDo
- private static void translateSideInputs(
- List<PCollectionView<?>> sideInputs,
- TranslationContext context) {
- Map<String, Object> nonParInputs = new HashMap<>();
-
- for (PCollectionView<?> view : sideInputs) {
- nonParInputs.put(
- view.getTagInternal().getId(),
- context.asOutputReference(view));
- }
-
- context.addInput(PropertyNames.NON_PARALLEL_INPUTS, nonParInputs);
- }
-
- private static void translateFn(
- DoFn fn,
- WindowingStrategy windowingStrategy,
- Iterable<PCollectionView<?>> sideInputs,
- Coder inputCoder,
- TranslationContext context) {
- context.addInput(PropertyNames.USER_FN, fn.getClass().getName());
- context.addInput(
- PropertyNames.SERIALIZED_FN,
- byteArrayToJsonString(serializeToByteArray(
- new DoFnInfo(fn, windowingStrategy, sideInputs, inputCoder))));
- }
-
- private static void translateOutputs(
- PCollectionTuple outputs,
- TranslationContext context) {
- for (Map.Entry<TupleTag<?>, PCollection<?>> entry
- : outputs.getAll().entrySet()) {
- TupleTag<?> tag = entry.getKey();
- PCollection<?> output = entry.getValue();
- context.addOutput(tag.getId(), output);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java
deleted file mode 100644
index c1f0e74..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowServiceException.java
+++ /dev/null
@@ -1,33 +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.runners;
-
-import javax.annotation.Nullable;
-
-/**
- * Signals there was an error retrieving information about a job from the Cloud Dataflow Service.
- */
-public class DataflowServiceException extends DataflowJobException {
- DataflowServiceException(DataflowPipelineJob job, String message) {
- this(job, message, null);
- }
-
- DataflowServiceException(DataflowPipelineJob job, String message, @Nullable Throwable cause) {
- super(job, message, cause);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java
deleted file mode 100644
index b068f50..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/AssignWindows.java
+++ /dev/null
@@ -1,89 +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.runners.dataflow;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * A primitive {@link PTransform} that implements the {@link Window#into(WindowFn)}
- * {@link PTransform}.
- *
- * For an application of {@link Window#into(WindowFn)} that changes the {@link WindowFn}, applies
- * a primitive {@link PTransform} in the Dataflow service.
- *
- * For an application of {@link Window#into(WindowFn)} that does not change the {@link WindowFn},
- * applies an identity {@link ParDo} and sets the windowing strategy of the output
- * {@link PCollection}.
- *
- * For internal use only.
- *
- * @param <T> the type of input element
- */
-public class AssignWindows<T> extends PTransform<PCollection<T>, PCollection<T>> {
- private final Window.Bound<T> transform;
-
- /**
- * Builds an instance of this class from the overriden transform.
- */
- @SuppressWarnings("unused") // Used via reflection
- public AssignWindows(Window.Bound<T> transform) {
- this.transform = transform;
- }
-
- @Override
- public PCollection<T> apply(PCollection<T> input) {
- WindowingStrategy<?, ?> outputStrategy =
- transform.getOutputStrategyInternal(input.getWindowingStrategy());
- if (transform.getWindowFn() != null) {
- // If the windowFn changed, we create a primitive, and run the AssignWindows operation here.
- return PCollection.<T>createPrimitiveOutputInternal(
- input.getPipeline(), outputStrategy, input.isBounded());
- } else {
- // If the windowFn didn't change, we just run a pass-through transform and then set the
- // new windowing strategy.
- return input.apply(ParDo.named("Identity").of(new DoFn<T, T>() {
- @Override
- public void processElement(DoFn<T, T>.ProcessContext c) throws Exception {
- c.output(c.element());
- }
- })).setWindowingStrategyInternal(outputStrategy);
- }
- }
-
- @Override
- public void validate(PCollection<T> input) {
- transform.validate(input);
- }
-
- @Override
- protected Coder<?> getDefaultOutputCoder(PCollection<T> input) {
- return input.getCoder();
- }
-
- @Override
- protected String getKindString() {
- return "Window.Into()";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.java
deleted file mode 100755
index b0952a6..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/BigQueryIOTranslator.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.sdk.runners.dataflow;
-
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.runners.DataflowPipelineTranslator;
-import org.apache.beam.sdk.util.PropertyNames;
-
-import com.google.api.services.bigquery.model.TableReference;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * BigQuery transform support code for the Dataflow backend.
- */
-public class BigQueryIOTranslator {
- private static final Logger LOG = LoggerFactory.getLogger(BigQueryIOTranslator.class);
-
- /**
- * Implements BigQueryIO Read translation for the Dataflow backend.
- */
- public static class ReadTranslator
- implements DataflowPipelineTranslator.TransformTranslator<BigQueryIO.Read.Bound> {
-
- @Override
- public void translate(
- BigQueryIO.Read.Bound transform, DataflowPipelineTranslator.TranslationContext context) {
- // Actual translation.
- context.addStep(transform, "ParallelRead");
- context.addInput(PropertyNames.FORMAT, "bigquery");
- context.addInput(PropertyNames.BIGQUERY_EXPORT_FORMAT, "FORMAT_AVRO");
-
- if (transform.getQuery() != null) {
- context.addInput(PropertyNames.BIGQUERY_QUERY, transform.getQuery());
- context.addInput(PropertyNames.BIGQUERY_FLATTEN_RESULTS, transform.getFlattenResults());
- } else {
- TableReference table = transform.getTable();
- if (table.getProjectId() == null) {
- // If user does not specify a project we assume the table to be located in the project
- // that owns the Dataflow job.
- String projectIdFromOptions = context.getPipelineOptions().getProject();
- LOG.warn(String.format(BigQueryIO.SET_PROJECT_FROM_OPTIONS_WARNING, table.getDatasetId(),
- table.getDatasetId(), table.getTableId(), projectIdFromOptions));
- table.setProjectId(projectIdFromOptions);
- }
-
- context.addInput(PropertyNames.BIGQUERY_TABLE, table.getTableId());
- context.addInput(PropertyNames.BIGQUERY_DATASET, table.getDatasetId());
- if (table.getProjectId() != null) {
- context.addInput(PropertyNames.BIGQUERY_PROJECT, table.getProjectId());
- }
- }
- context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java
deleted file mode 100755
index a3f9fda..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/CustomSources.java
+++ /dev/null
@@ -1,121 +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.runners.dataflow;
-
-import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray;
-import static org.apache.beam.sdk.util.Structs.addString;
-import static org.apache.beam.sdk.util.Structs.addStringList;
-
-import static com.google.api.client.util.Base64.encodeBase64String;
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.io.Source;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.util.CloudObject;
-
-import com.google.api.services.dataflow.model.SourceMetadata;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ByteString;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-
-
-/**
- * A helper class for supporting sources defined as {@code Source}.
- *
- * <p>Provides a bridge between the high-level {@code Source} API and the
- * low-level {@code CloudSource} class.
- */
-public class CustomSources {
- private static final String SERIALIZED_SOURCE = "serialized_source";
- @VisibleForTesting static final String SERIALIZED_SOURCE_SPLITS = "serialized_source_splits";
- /**
- * The current limit on the size of a ReportWorkItemStatus RPC to Google Cloud Dataflow, which
- * includes the initial splits, is 20 MB.
- */
- public static final long DATAFLOW_SPLIT_RESPONSE_API_SIZE_BYTES = 20 * (1 << 20);
-
- private static final Logger LOG = LoggerFactory.getLogger(CustomSources.class);
-
- private static final ByteString firstSplitKey = ByteString.copyFromUtf8("0000000000000001");
-
- public static boolean isFirstUnboundedSourceSplit(ByteString splitKey) {
- return splitKey.equals(firstSplitKey);
- }
-
- private static int getDesiredNumUnboundedSourceSplits(DataflowPipelineOptions options) {
- if (options.getMaxNumWorkers() > 0) {
- return options.getMaxNumWorkers();
- } else if (options.getNumWorkers() > 0) {
- return options.getNumWorkers() * 3;
- } else {
- return 20;
- }
- }
-
- public static com.google.api.services.dataflow.model.Source serializeToCloudSource(
- Source<?> source, PipelineOptions options) throws Exception {
- com.google.api.services.dataflow.model.Source cloudSource =
- new com.google.api.services.dataflow.model.Source();
- // We ourselves act as the SourceFormat.
- cloudSource.setSpec(CloudObject.forClass(CustomSources.class));
- addString(
- cloudSource.getSpec(), SERIALIZED_SOURCE, encodeBase64String(serializeToByteArray(source)));
-
- SourceMetadata metadata = new SourceMetadata();
- if (source instanceof BoundedSource) {
- BoundedSource<?> boundedSource = (BoundedSource<?>) source;
- try {
- metadata.setProducesSortedKeys(boundedSource.producesSortedKeys(options));
- } catch (Exception e) {
- LOG.warn("Failed to check if the source produces sorted keys: " + source, e);
- }
-
- // Size estimation is best effort so we continue even if it fails here.
- try {
- metadata.setEstimatedSizeBytes(boundedSource.getEstimatedSizeBytes(options));
- } catch (Exception e) {
- LOG.warn("Size estimation of the source failed: " + source, e);
- }
- } else if (source instanceof UnboundedSource) {
- UnboundedSource<?, ?> unboundedSource = (UnboundedSource<?, ?>) source;
- metadata.setInfinite(true);
- List<String> encodedSplits = new ArrayList<>();
- int desiredNumSplits =
- getDesiredNumUnboundedSourceSplits(options.as(DataflowPipelineOptions.class));
- for (UnboundedSource<?, ?> split :
- unboundedSource.generateInitialSplits(desiredNumSplits, options)) {
- encodedSplits.add(encodeBase64String(serializeToByteArray(split)));
- }
- checkArgument(!encodedSplits.isEmpty(), "UnboundedSources must have at least one split");
- addStringList(cloudSource.getSpec(), SERIALIZED_SOURCE_SPLITS, encodedSplits);
- } else {
- throw new IllegalArgumentException("Unexpected source kind: " + source.getClass());
- }
-
- cloudSource.setMetadata(metadata);
- return cloudSource;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java
deleted file mode 100755
index 088ce63..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowAggregatorTransforms.java
+++ /dev/null
@@ -1,81 +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.runners.dataflow;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-
-import com.google.common.collect.BiMap;
-import com.google.common.collect.HashBiMap;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Map;
-
-/**
- * A mapping relating {@link Aggregator}s and the {@link PTransform} in which they are used.
- */
-public class DataflowAggregatorTransforms {
- private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorTransforms;
- private final Multimap<PTransform<?, ?>, AppliedPTransform<?, ?, ?>> transformAppliedTransforms;
- private final BiMap<AppliedPTransform<?, ?, ?>, String> appliedStepNames;
-
- public DataflowAggregatorTransforms(
- Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorTransforms,
- Map<AppliedPTransform<?, ?, ?>, String> transformStepNames) {
- this.aggregatorTransforms = aggregatorTransforms;
- appliedStepNames = HashBiMap.create(transformStepNames);
-
- transformAppliedTransforms = HashMultimap.create();
- for (AppliedPTransform<?, ?, ?> appliedTransform : transformStepNames.keySet()) {
- transformAppliedTransforms.put(appliedTransform.getTransform(), appliedTransform);
- }
- }
-
- /**
- * Returns true if the provided {@link Aggregator} is used in the constructing {@link Pipeline}.
- */
- public boolean contains(Aggregator<?, ?> aggregator) {
- return aggregatorTransforms.containsKey(aggregator);
- }
-
- /**
- * Gets the step names in which the {@link Aggregator} is used.
- */
- public Collection<String> getAggregatorStepNames(Aggregator<?, ?> aggregator) {
- Collection<String> names = new HashSet<>();
- Collection<PTransform<?, ?>> transforms = aggregatorTransforms.get(aggregator);
- for (PTransform<?, ?> transform : transforms) {
- for (AppliedPTransform<?, ?, ?> applied : transformAppliedTransforms.get(transform)) {
- names.add(appliedStepNames.get(applied));
- }
- }
- return names;
- }
-
- /**
- * Gets the {@link PTransform} that was assigned the provided step name.
- */
- public AppliedPTransform<?, ?, ?> getAppliedTransformForStepName(String stepName) {
- return appliedStepNames.inverse().get(stepName);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
deleted file mode 100755
index 35ef229..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/DataflowMetricUpdateExtractor.java
+++ /dev/null
@@ -1,111 +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.runners.dataflow;
-
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.PTransform;
-
-import com.google.api.services.dataflow.model.MetricStructuredName;
-import com.google.api.services.dataflow.model.MetricUpdate;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Methods for extracting the values of an {@link Aggregator} from a collection of {@link
- * MetricUpdate MetricUpdates}.
- */
-public final class DataflowMetricUpdateExtractor {
- private static final String STEP_NAME_CONTEXT_KEY = "step";
- private static final String IS_TENTATIVE_KEY = "tentative";
-
- private DataflowMetricUpdateExtractor() {
- // Do not instantiate.
- }
-
- /**
- * Extract the values of the provided {@link Aggregator} at each {@link PTransform} it was used in
- * according to the provided {@link DataflowAggregatorTransforms} from the given list of {@link
- * MetricUpdate MetricUpdates}.
- */
- public static <OutputT> Map<String, OutputT> fromMetricUpdates(Aggregator<?, OutputT> aggregator,
- DataflowAggregatorTransforms aggregatorTransforms, List<MetricUpdate> metricUpdates) {
- Map<String, OutputT> results = new HashMap<>();
- if (metricUpdates == null) {
- return results;
- }
-
- String aggregatorName = aggregator.getName();
- Collection<String> aggregatorSteps = aggregatorTransforms.getAggregatorStepNames(aggregator);
-
- for (MetricUpdate metricUpdate : metricUpdates) {
- MetricStructuredName metricStructuredName = metricUpdate.getName();
- Map<String, String> context = metricStructuredName.getContext();
- if (metricStructuredName.getName().equals(aggregatorName) && context != null
- && aggregatorSteps.contains(context.get(STEP_NAME_CONTEXT_KEY))) {
- AppliedPTransform<?, ?, ?> transform =
- aggregatorTransforms.getAppliedTransformForStepName(
- context.get(STEP_NAME_CONTEXT_KEY));
- String fullName = transform.getFullName();
- // Prefer the tentative (fresher) value if it exists.
- if (Boolean.parseBoolean(context.get(IS_TENTATIVE_KEY)) || !results.containsKey(fullName)) {
- results.put(fullName, toValue(aggregator, metricUpdate));
- }
- }
- }
-
- return results;
-
- }
-
- private static <OutputT> OutputT toValue(
- Aggregator<?, OutputT> aggregator, MetricUpdate metricUpdate) {
- CombineFn<?, ?, OutputT> combineFn = aggregator.getCombineFn();
- Class<? super OutputT> outputType = combineFn.getOutputType().getRawType();
-
- if (outputType.equals(Long.class)) {
- @SuppressWarnings("unchecked")
- OutputT asLong = (OutputT) Long.valueOf(toNumber(metricUpdate).longValue());
- return asLong;
- }
- if (outputType.equals(Integer.class)) {
- @SuppressWarnings("unchecked")
- OutputT asInt = (OutputT) Integer.valueOf(toNumber(metricUpdate).intValue());
- return asInt;
- }
- if (outputType.equals(Double.class)) {
- @SuppressWarnings("unchecked")
- OutputT asDouble = (OutputT) Double.valueOf(toNumber(metricUpdate).doubleValue());
- return asDouble;
- }
- throw new UnsupportedOperationException(
- "Unsupported Output Type " + outputType + " in aggregator " + aggregator);
- }
-
- private static Number toNumber(MetricUpdate update) {
- if (update.getScalar() instanceof Number) {
- return (Number) update.getScalar();
- }
- throw new IllegalArgumentException(
- "Metric Update " + update + " does not have a numeric scalar");
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java
deleted file mode 100755
index e299bec..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/PubsubIOTranslator.java
+++ /dev/null
@@ -1,108 +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.runners.dataflow;
-
-import org.apache.beam.sdk.io.PubsubIO;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
-import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
-import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import org.apache.beam.sdk.util.PropertyNames;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * Pubsub transform support code for the Dataflow backend.
- */
-public class PubsubIOTranslator {
-
- /**
- * Implements PubsubIO Read translation for the Dataflow backend.
- */
- public static class ReadTranslator<T> implements TransformTranslator<PubsubIO.Read.Bound<T>> {
- @Override
- @SuppressWarnings({"rawtypes", "unchecked"})
- public void translate(
- PubsubIO.Read.Bound transform,
- TranslationContext context) {
- translateReadHelper(transform, context);
- }
-
- private <T> void translateReadHelper(
- PubsubIO.Read.Bound<T> transform,
- TranslationContext context) {
- if (!context.getPipelineOptions().isStreaming()) {
- throw new IllegalArgumentException(
- "PubsubIO.Read can only be used with the Dataflow streaming runner.");
- }
-
- context.addStep(transform, "ParallelRead");
- context.addInput(PropertyNames.FORMAT, "pubsub");
- if (transform.getTopic() != null) {
- context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic().asV1Beta1Path());
- }
- if (transform.getSubscription() != null) {
- context.addInput(
- PropertyNames.PUBSUB_SUBSCRIPTION, transform.getSubscription().asV1Beta1Path());
- }
- if (transform.getTimestampLabel() != null) {
- context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel());
- }
- if (transform.getIdLabel() != null) {
- context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel());
- }
- context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
- }
- }
-
- /**
- * Implements PubsubIO Write translation for the Dataflow backend.
- */
- public static class WriteTranslator<T>
- implements TransformTranslator<DataflowPipelineRunner.StreamingPubsubIOWrite<T>> {
-
- @Override
- @SuppressWarnings({"rawtypes", "unchecked"})
- public void translate(
- DataflowPipelineRunner.StreamingPubsubIOWrite transform,
- TranslationContext context) {
- translateWriteHelper(transform, context);
- }
-
- private <T> void translateWriteHelper(
- DataflowPipelineRunner.StreamingPubsubIOWrite<T> customTransform,
- TranslationContext context) {
- if (!context.getPipelineOptions().isStreaming()) {
- throw new IllegalArgumentException(
- "PubsubIO.Write is non-primitive for the Dataflow batch runner.");
- }
-
- PubsubIO.Write.Bound<T> transform = customTransform.getOverriddenTransform();
-
- context.addStep(customTransform, "ParallelWrite");
- context.addInput(PropertyNames.FORMAT, "pubsub");
- context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic().asV1Beta1Path());
- if (transform.getTimestampLabel() != null) {
- context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel());
- }
- if (transform.getIdLabel() != null) {
- context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel());
- }
- context.addEncodingInput(WindowedValue.getValueOnlyCoder(transform.getCoder()));
- context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(customTransform));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java
deleted file mode 100755
index 0c394a1..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/ReadTranslator.java
+++ /dev/null
@@ -1,105 +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.runners.dataflow;
-
-import static org.apache.beam.sdk.util.Structs.addBoolean;
-import static org.apache.beam.sdk.util.Structs.addDictionary;
-import static org.apache.beam.sdk.util.Structs.addLong;
-
-import org.apache.beam.sdk.io.FileBasedSource;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.Source;
-import org.apache.beam.sdk.runners.DataflowPipelineTranslator;
-import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
-import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.PropertyNames;
-import org.apache.beam.sdk.values.PValue;
-
-import com.google.api.services.dataflow.model.SourceMetadata;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Translator for the {@code Read} {@code PTransform} for the Dataflow back-end.
- */
-public class ReadTranslator implements TransformTranslator<Read.Bounded<?>> {
- @Override
- public void translate(Read.Bounded<?> transform, TranslationContext context) {
- translateReadHelper(transform.getSource(), transform, context);
- }
-
- public static <T> void translateReadHelper(Source<T> source,
- PTransform<?, ? extends PValue> transform,
- DataflowPipelineTranslator.TranslationContext context) {
- try {
- // TODO: Move this validation out of translation once IOChannelUtils is portable
- // and can be reconstructed on the worker.
- if (source instanceof FileBasedSource) {
- String filePatternOrSpec = ((FileBasedSource<?>) source).getFileOrPatternSpec();
- context.getPipelineOptions()
- .getPathValidator()
- .validateInputFilePatternSupported(filePatternOrSpec);
- }
-
- context.addStep(transform, "ParallelRead");
- context.addInput(PropertyNames.FORMAT, PropertyNames.CUSTOM_SOURCE_FORMAT);
- context.addInput(
- PropertyNames.SOURCE_STEP_INPUT,
- cloudSourceToDictionary(
- CustomSources.serializeToCloudSource(source, context.getPipelineOptions())));
- context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-
- // Represents a cloud Source as a dictionary for encoding inside the {@code SOURCE_STEP_INPUT}
- // property of CloudWorkflowStep.input.
- private static Map<String, Object> cloudSourceToDictionary(
- com.google.api.services.dataflow.model.Source source) {
- // Do not translate encoding - the source's encoding is translated elsewhere
- // to the step's output info.
- Map<String, Object> res = new HashMap<>();
- addDictionary(res, PropertyNames.SOURCE_SPEC, source.getSpec());
- if (source.getMetadata() != null) {
- addDictionary(res, PropertyNames.SOURCE_METADATA,
- cloudSourceMetadataToDictionary(source.getMetadata()));
- }
- if (source.getDoesNotNeedSplitting() != null) {
- addBoolean(
- res, PropertyNames.SOURCE_DOES_NOT_NEED_SPLITTING, source.getDoesNotNeedSplitting());
- }
- return res;
- }
-
- private static Map<String, Object> cloudSourceMetadataToDictionary(SourceMetadata metadata) {
- Map<String, Object> res = new HashMap<>();
- if (metadata.getProducesSortedKeys() != null) {
- addBoolean(res, PropertyNames.SOURCE_PRODUCES_SORTED_KEYS, metadata.getProducesSortedKeys());
- }
- if (metadata.getEstimatedSizeBytes() != null) {
- addLong(res, PropertyNames.SOURCE_ESTIMATED_SIZE_BYTES, metadata.getEstimatedSizeBytes());
- }
- if (metadata.getInfinite() != null) {
- addBoolean(res, PropertyNames.SOURCE_IS_INFINITE, metadata.getInfinite());
- }
- return res;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java
deleted file mode 100755
index 3630ea7..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/dataflow/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Implementation of the {@link org.apache.beam.sdk.runners.DataflowPipelineRunner}.
- */
-package org.apache.beam.sdk.runners.dataflow;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java
deleted file mode 100644
index f8913af..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineOptions.java
+++ /dev/null
@@ -1,28 +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.testing;
-
-import org.apache.beam.sdk.options.BlockingDataflowPipelineOptions;
-
-/**
- * A set of options used to configure the {@link TestPipeline}.
- */
-public interface TestDataflowPipelineOptions extends TestPipelineOptions,
- BlockingDataflowPipelineOptions {
-
-}
[03/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
deleted file mode 100644
index 27c0acc..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/DataflowPipelineTranslatorTest.java
+++ /dev/null
@@ -1,965 +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.runners;
-
-import static org.apache.beam.sdk.util.Structs.addObject;
-import static org.apache.beam.sdk.util.Structs.getDictionary;
-import static org.apache.beam.sdk.util.Structs.getString;
-
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.hasKey;
-import static org.hamcrest.core.IsInstanceOf.instanceOf;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.argThat;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.DataflowPipelineWorkerPoolOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import org.apache.beam.sdk.transforms.Count;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.util.GcsUtil;
-import org.apache.beam.sdk.util.OutputReference;
-import org.apache.beam.sdk.util.PropertyNames;
-import org.apache.beam.sdk.util.Structs;
-import org.apache.beam.sdk.util.TestCredential;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.TupleTag;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.Step;
-import com.google.api.services.dataflow.model.WorkerPool;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-
-import org.hamcrest.Matchers;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.internal.matchers.ThrowableMessageMatcher;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.ArgumentMatcher;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Tests for DataflowPipelineTranslator.
- */
-@RunWith(JUnit4.class)
-public class DataflowPipelineTranslatorTest implements Serializable {
-
- @Rule public transient ExpectedException thrown = ExpectedException.none();
-
- // A Custom Mockito matcher for an initial Job that checks that all
- // expected fields are set.
- private static class IsValidCreateRequest extends ArgumentMatcher<Job> {
- @Override
- public boolean matches(Object o) {
- Job job = (Job) o;
- return job.getId() == null
- && job.getProjectId() == null
- && job.getName() != null
- && job.getType() != null
- && job.getEnvironment() != null
- && job.getSteps() != null
- && job.getCurrentState() == null
- && job.getCurrentStateTime() == null
- && job.getExecutionInfo() == null
- && job.getCreateTime() == null;
- }
- }
-
- private Pipeline buildPipeline(DataflowPipelineOptions options) {
- options.setRunner(DataflowPipelineRunner.class);
- Pipeline p = Pipeline.create(options);
-
- p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
- .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
-
- return p;
- }
-
- private static Dataflow buildMockDataflow(
- ArgumentMatcher<Job> jobMatcher) throws IOException {
- Dataflow mockDataflowClient = mock(Dataflow.class);
- Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
- Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
- Dataflow.Projects.Jobs.Create mockRequest = mock(
- Dataflow.Projects.Jobs.Create.class);
-
- when(mockDataflowClient.projects()).thenReturn(mockProjects);
- when(mockProjects.jobs()).thenReturn(mockJobs);
- when(mockJobs.create(eq("someProject"), argThat(jobMatcher)))
- .thenReturn(mockRequest);
-
- Job resultJob = new Job();
- resultJob.setId("newid");
- when(mockRequest.execute()).thenReturn(resultJob);
- return mockDataflowClient;
- }
-
- private static DataflowPipelineOptions buildPipelineOptions() throws IOException {
- GcsUtil mockGcsUtil = mock(GcsUtil.class);
- when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer<List<GcsPath>>() {
- @Override
- public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
- return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
- }
- });
- when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(true);
- when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
-
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setGcpCredential(new TestCredential());
- options.setJobName("some-job-name");
- options.setProject("some-project");
- options.setTempLocation(GcsPath.fromComponents("somebucket", "some/path").toString());
- options.setFilesToStage(new LinkedList<String>());
- options.setDataflowClient(buildMockDataflow(new IsValidCreateRequest()));
- options.setGcsUtil(mockGcsUtil);
- return options;
- }
-
- @Test
- public void testSettingOfSdkPipelineOptions() throws IOException {
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setRunner(DataflowPipelineRunner.class);
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- // Note that the contents of this materialized map may be changed by the act of reading an
- // option, which will cause the default to get materialized whereas it would otherwise be
- // left absent. It is permissible to simply alter this test to reflect current behavior.
- Map<String, Object> settings = new HashMap<>();
- settings.put("appName", "DataflowPipelineTranslatorTest");
- settings.put("project", "some-project");
- settings.put("pathValidatorClass", "org.apache.beam.sdk.util.DataflowPathValidator");
- settings.put("runner", "org.apache.beam.sdk.runners.DataflowPipelineRunner");
- settings.put("jobName", "some-job-name");
- settings.put("tempLocation", "gs://somebucket/some/path");
- settings.put("stagingLocation", "gs://somebucket/some/path/staging");
- settings.put("stableUniqueNames", "WARNING");
- settings.put("streaming", false);
- settings.put("numberOfWorkerHarnessThreads", 0);
- settings.put("experiments", null);
-
- Map<String, Object> sdkPipelineOptions = job.getEnvironment().getSdkPipelineOptions();
- assertThat(sdkPipelineOptions, hasKey("options"));
- assertEquals(settings, sdkPipelineOptions.get("options"));
- }
-
- @Test
- public void testNetworkConfig() throws IOException {
- final String testNetwork = "test-network";
-
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setNetwork(testNetwork);
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(1, job.getEnvironment().getWorkerPools().size());
- assertEquals(testNetwork,
- job.getEnvironment().getWorkerPools().get(0).getNetwork());
- }
-
- @Test
- public void testNetworkConfigMissing() throws IOException {
- DataflowPipelineOptions options = buildPipelineOptions();
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(1, job.getEnvironment().getWorkerPools().size());
- assertNull(job.getEnvironment().getWorkerPools().get(0).getNetwork());
- }
-
- @Test
- public void testSubnetworkConfig() throws IOException {
- final String testSubnetwork = "regions/REGION/subnetworks/SUBNETWORK";
-
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setSubnetwork(testSubnetwork);
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(1, job.getEnvironment().getWorkerPools().size());
- assertEquals(testSubnetwork,
- job.getEnvironment().getWorkerPools().get(0).getSubnetwork());
- }
-
- @Test
- public void testSubnetworkConfigMissing() throws IOException {
- DataflowPipelineOptions options = buildPipelineOptions();
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(1, job.getEnvironment().getWorkerPools().size());
- assertNull(job.getEnvironment().getWorkerPools().get(0).getSubnetwork());
- }
-
- @Test
- public void testScalingAlgorithmMissing() throws IOException {
- DataflowPipelineOptions options = buildPipelineOptions();
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(1, job.getEnvironment().getWorkerPools().size());
- // Autoscaling settings are always set.
- assertNull(
- job
- .getEnvironment()
- .getWorkerPools()
- .get(0)
- .getAutoscalingSettings()
- .getAlgorithm());
- assertEquals(
- 0,
- job
- .getEnvironment()
- .getWorkerPools()
- .get(0)
- .getAutoscalingSettings()
- .getMaxNumWorkers()
- .intValue());
- }
-
- @Test
- public void testScalingAlgorithmNone() throws IOException {
- final DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType noScaling =
- DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE;
-
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setAutoscalingAlgorithm(noScaling);
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(1, job.getEnvironment().getWorkerPools().size());
- assertEquals(
- "AUTOSCALING_ALGORITHM_NONE",
- job
- .getEnvironment()
- .getWorkerPools()
- .get(0)
- .getAutoscalingSettings()
- .getAlgorithm());
- assertEquals(
- 0,
- job
- .getEnvironment()
- .getWorkerPools()
- .get(0)
- .getAutoscalingSettings()
- .getMaxNumWorkers()
- .intValue());
- }
-
- @Test
- public void testMaxNumWorkersIsPassedWhenNoAlgorithmIsSet() throws IOException {
- final DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType noScaling = null;
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setMaxNumWorkers(42);
- options.setAutoscalingAlgorithm(noScaling);
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(1, job.getEnvironment().getWorkerPools().size());
- assertNull(
- job
- .getEnvironment()
- .getWorkerPools()
- .get(0)
- .getAutoscalingSettings()
- .getAlgorithm());
- assertEquals(
- 42,
- job
- .getEnvironment()
- .getWorkerPools()
- .get(0)
- .getAutoscalingSettings()
- .getMaxNumWorkers()
- .intValue());
- }
-
- @Test
- public void testZoneConfig() throws IOException {
- final String testZone = "test-zone-1";
-
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setZone(testZone);
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(1, job.getEnvironment().getWorkerPools().size());
- assertEquals(testZone,
- job.getEnvironment().getWorkerPools().get(0).getZone());
- }
-
- @Test
- public void testWorkerMachineTypeConfig() throws IOException {
- final String testMachineType = "test-machine-type";
-
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setWorkerMachineType(testMachineType);
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(1, job.getEnvironment().getWorkerPools().size());
-
- WorkerPool workerPool = job.getEnvironment().getWorkerPools().get(0);
- assertEquals(testMachineType, workerPool.getMachineType());
- }
-
- @Test
- public void testDiskSizeGbConfig() throws IOException {
- final Integer diskSizeGb = 1234;
-
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setDiskSizeGb(diskSizeGb);
-
- Pipeline p = buildPipeline(options);
- p.traverseTopologically(new RecordingPipelineVisitor());
- Job job =
- DataflowPipelineTranslator.fromOptions(options)
- .translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(1, job.getEnvironment().getWorkerPools().size());
- assertEquals(diskSizeGb,
- job.getEnvironment().getWorkerPools().get(0).getDiskSizeGb());
- }
-
- @Test
- public void testPredefinedAddStep() throws Exception {
- DataflowPipelineOptions options = buildPipelineOptions();
-
- DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
- DataflowPipelineTranslator.registerTransformTranslator(
- EmbeddedTransform.class, new EmbeddedTranslator());
-
- // Create a predefined step using another pipeline
- Step predefinedStep = createPredefinedStep();
-
- // Create a pipeline that the predefined step will be embedded into
- Pipeline pipeline = Pipeline.create(options);
- pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in"))
- .apply(ParDo.of(new NoOpFn()))
- .apply(new EmbeddedTransform(predefinedStep.clone()))
- .apply(ParDo.of(new NoOpFn()));
- Job job =
- translator
- .translate(
- pipeline,
- (DataflowPipelineRunner) pipeline.getRunner(),
- Collections.<DataflowPackage>emptyList())
- .getJob();
-
- List<Step> steps = job.getSteps();
- assertEquals(4, steps.size());
-
- // The input to the embedded step should match the output of the step before
- Map<String, Object> step1Out = getOutputPortReference(steps.get(1));
- Map<String, Object> step2In = getDictionary(
- steps.get(2).getProperties(), PropertyNames.PARALLEL_INPUT);
- assertEquals(step1Out, step2In);
-
- // The output from the embedded step should match the input of the step after
- Map<String, Object> step2Out = getOutputPortReference(steps.get(2));
- Map<String, Object> step3In = getDictionary(
- steps.get(3).getProperties(), PropertyNames.PARALLEL_INPUT);
- assertEquals(step2Out, step3In);
-
- // The step should not have been modified other than remapping the input
- Step predefinedStepClone = predefinedStep.clone();
- Step embeddedStepClone = steps.get(2).clone();
- predefinedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT);
- embeddedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT);
- assertEquals(predefinedStepClone, embeddedStepClone);
- }
-
- /**
- * Construct a OutputReference for the output of the step.
- */
- private static OutputReference getOutputPortReference(Step step) throws Exception {
- // TODO: This should be done via a Structs accessor.
- @SuppressWarnings("unchecked")
- List<Map<String, Object>> output =
- (List<Map<String, Object>>) step.getProperties().get(PropertyNames.OUTPUT_INFO);
- String outputTagId = getString(Iterables.getOnlyElement(output), PropertyNames.OUTPUT_NAME);
- return new OutputReference(step.getName(), outputTagId);
- }
-
- /**
- * Returns a Step for a DoFn by creating and translating a pipeline.
- */
- private static Step createPredefinedStep() throws Exception {
- DataflowPipelineOptions options = buildPipelineOptions();
- DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
- Pipeline pipeline = Pipeline.create(options);
- String stepName = "DoFn1";
- pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in"))
- .apply(ParDo.of(new NoOpFn()).named(stepName))
- .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/out"));
- Job job =
- translator
- .translate(
- pipeline,
- (DataflowPipelineRunner) pipeline.getRunner(),
- Collections.<DataflowPackage>emptyList())
- .getJob();
-
- assertEquals(13, job.getSteps().size());
- Step step = job.getSteps().get(1);
- assertEquals(stepName, getString(step.getProperties(), PropertyNames.USER_NAME));
- return step;
- }
-
- private static class NoOpFn extends DoFn<String, String> {
- @Override public void processElement(ProcessContext c) throws Exception {
- c.output(c.element());
- }
- }
-
- /**
- * A placeholder transform that will be used to substitute a predefined Step.
- */
- private static class EmbeddedTransform
- extends PTransform<PCollection<String>, PCollection<String>> {
- private final Step step;
-
- public EmbeddedTransform(Step step) {
- this.step = step;
- }
-
- @Override
- public PCollection<String> apply(PCollection<String> input) {
- return PCollection.createPrimitiveOutputInternal(
- input.getPipeline(),
- WindowingStrategy.globalDefault(),
- input.isBounded());
- }
-
- @Override
- protected Coder<?> getDefaultOutputCoder() {
- return StringUtf8Coder.of();
- }
- }
-
- /**
- * A TransformTranslator that adds the predefined Step using
- * {@link TranslationContext#addStep} and remaps the input port reference.
- */
- private static class EmbeddedTranslator
- implements DataflowPipelineTranslator.TransformTranslator<EmbeddedTransform> {
- @Override public void translate(EmbeddedTransform transform, TranslationContext context) {
- addObject(transform.step.getProperties(), PropertyNames.PARALLEL_INPUT,
- context.asOutputReference(context.getInput(transform)));
- context.addStep(transform, transform.step);
- }
- }
-
- /**
- * A composite transform that returns an output that is unrelated to
- * the input.
- */
- private static class UnrelatedOutputCreator
- extends PTransform<PCollection<Integer>, PCollection<Integer>> {
-
- @Override
- public PCollection<Integer> apply(PCollection<Integer> input) {
- // Apply an operation so that this is a composite transform.
- input.apply(Count.<Integer>perElement());
-
- // Return a value unrelated to the input.
- return input.getPipeline().apply(Create.of(1, 2, 3, 4));
- }
-
- @Override
- protected Coder<?> getDefaultOutputCoder() {
- return VarIntCoder.of();
- }
- }
-
- /**
- * A composite transform that returns an output that is unbound.
- */
- private static class UnboundOutputCreator
- extends PTransform<PCollection<Integer>, PDone> {
-
- @Override
- public PDone apply(PCollection<Integer> input) {
- // Apply an operation so that this is a composite transform.
- input.apply(Count.<Integer>perElement());
-
- return PDone.in(input.getPipeline());
- }
-
- @Override
- protected Coder<?> getDefaultOutputCoder() {
- return VoidCoder.of();
- }
- }
-
- /**
- * A composite transform that returns a partially bound output.
- *
- * <p>This is not allowed and will result in a failure.
- */
- private static class PartiallyBoundOutputCreator
- extends PTransform<PCollection<Integer>, PCollectionTuple> {
-
- public final TupleTag<Integer> sumTag = new TupleTag<>("sum");
- public final TupleTag<Void> doneTag = new TupleTag<>("done");
-
- @Override
- public PCollectionTuple apply(PCollection<Integer> input) {
- PCollection<Integer> sum = input.apply(Sum.integersGlobally());
-
- // Fails here when attempting to construct a tuple with an unbound object.
- return PCollectionTuple.of(sumTag, sum)
- .and(doneTag, PCollection.<Void>createPrimitiveOutputInternal(
- input.getPipeline(),
- WindowingStrategy.globalDefault(),
- input.isBounded()));
- }
- }
-
- @Test
- public void testMultiGraphPipelineSerialization() throws IOException {
- Pipeline p = Pipeline.create(buildPipelineOptions());
-
- PCollection<Integer> input = p.begin()
- .apply(Create.of(1, 2, 3));
-
- input.apply(new UnrelatedOutputCreator());
- input.apply(new UnboundOutputCreator());
-
- DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(
- PipelineOptionsFactory.as(DataflowPipelineOptions.class));
-
- // Check that translation doesn't fail.
- t.translate(
- p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
- }
-
- @Test
- public void testPartiallyBoundFailure() throws IOException {
- Pipeline p = Pipeline.create(buildPipelineOptions());
-
- PCollection<Integer> input = p.begin()
- .apply(Create.of(1, 2, 3));
-
- thrown.expect(IllegalStateException.class);
- input.apply(new PartiallyBoundOutputCreator());
-
- Assert.fail("Failure expected from use of partially bound output");
- }
-
- /**
- * This tests a few corner cases that should not crash.
- */
- @Test
- public void testGoodWildcards() throws Exception {
- DataflowPipelineOptions options = buildPipelineOptions();
- Pipeline pipeline = Pipeline.create(options);
- DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options);
-
- applyRead(pipeline, "gs://bucket/foo");
- applyRead(pipeline, "gs://bucket/foo/");
- applyRead(pipeline, "gs://bucket/foo/*");
- applyRead(pipeline, "gs://bucket/foo/?");
- applyRead(pipeline, "gs://bucket/foo/[0-9]");
- applyRead(pipeline, "gs://bucket/foo/*baz*");
- applyRead(pipeline, "gs://bucket/foo/*baz?");
- applyRead(pipeline, "gs://bucket/foo/[0-9]baz?");
- applyRead(pipeline, "gs://bucket/foo/baz/*");
- applyRead(pipeline, "gs://bucket/foo/baz/*wonka*");
- applyRead(pipeline, "gs://bucket/foo/*baz/wonka*");
- applyRead(pipeline, "gs://bucket/foo*/baz");
- applyRead(pipeline, "gs://bucket/foo?/baz");
- applyRead(pipeline, "gs://bucket/foo[0-9]/baz");
-
- // Check that translation doesn't fail.
- t.translate(
- pipeline,
- (DataflowPipelineRunner) pipeline.getRunner(),
- Collections.<DataflowPackage>emptyList());
- }
-
- private void applyRead(Pipeline pipeline, String path) {
- pipeline.apply("Read(" + path + ")", TextIO.Read.from(path));
- }
-
- /**
- * Recursive wildcards are not supported.
- * This tests "**".
- */
- @Test
- public void testBadWildcardRecursive() throws Exception {
- DataflowPipelineOptions options = buildPipelineOptions();
- Pipeline pipeline = Pipeline.create(options);
- DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options);
-
- pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz"));
-
- // Check that translation does fail.
- thrown.expectCause(Matchers.allOf(
- instanceOf(IllegalArgumentException.class),
- ThrowableMessageMatcher.hasMessage(containsString("Unsupported wildcard usage"))));
- t.translate(
- pipeline,
- (DataflowPipelineRunner) pipeline.getRunner(),
- Collections.<DataflowPackage>emptyList());
- }
-
- @Test
- public void testToSingletonTranslation() throws Exception {
- // A "change detector" test that makes sure the translation
- // of getting a PCollectionView<T> does not change
- // in bad ways during refactor
-
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setExperiments(ImmutableList.of("disable_ism_side_input"));
- DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-
- Pipeline pipeline = Pipeline.create(options);
- pipeline.apply(Create.of(1))
- .apply(View.<Integer>asSingleton());
- Job job =
- translator
- .translate(
- pipeline,
- (DataflowPipelineRunner) pipeline.getRunner(),
- Collections.<DataflowPackage>emptyList())
- .getJob();
-
- List<Step> steps = job.getSteps();
- assertEquals(2, steps.size());
-
- Step createStep = steps.get(0);
- assertEquals("ParallelRead", createStep.getKind());
-
- Step collectionToSingletonStep = steps.get(1);
- assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
-
- }
-
- @Test
- public void testToIterableTranslation() throws Exception {
- // A "change detector" test that makes sure the translation
- // of getting a PCollectionView<Iterable<T>> does not change
- // in bad ways during refactor
-
- DataflowPipelineOptions options = buildPipelineOptions();
- options.setExperiments(ImmutableList.of("disable_ism_side_input"));
- DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-
- Pipeline pipeline = Pipeline.create(options);
- pipeline.apply(Create.of(1, 2, 3))
- .apply(View.<Integer>asIterable());
- Job job =
- translator
- .translate(
- pipeline,
- (DataflowPipelineRunner) pipeline.getRunner(),
- Collections.<DataflowPackage>emptyList())
- .getJob();
-
- List<Step> steps = job.getSteps();
- assertEquals(2, steps.size());
-
- Step createStep = steps.get(0);
- assertEquals("ParallelRead", createStep.getKind());
-
- Step collectionToSingletonStep = steps.get(1);
- assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
- }
-
- @Test
- public void testToSingletonTranslationWithIsmSideInput() throws Exception {
- // A "change detector" test that makes sure the translation
- // of getting a PCollectionView<T> does not change
- // in bad ways during refactor
-
- DataflowPipelineOptions options = buildPipelineOptions();
- DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-
- Pipeline pipeline = Pipeline.create(options);
- pipeline.apply(Create.of(1))
- .apply(View.<Integer>asSingleton());
- Job job =
- translator
- .translate(
- pipeline,
- (DataflowPipelineRunner) pipeline.getRunner(),
- Collections.<DataflowPackage>emptyList())
- .getJob();
-
- List<Step> steps = job.getSteps();
- assertEquals(5, steps.size());
-
- @SuppressWarnings("unchecked")
- List<Map<String, Object>> toIsmRecordOutputs =
- (List<Map<String, Object>>) steps.get(3).getProperties().get(PropertyNames.OUTPUT_INFO);
- assertTrue(
- Structs.getBoolean(Iterables.getOnlyElement(toIsmRecordOutputs), "use_indexed_format"));
-
- Step collectionToSingletonStep = steps.get(4);
- assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
- }
-
- @Test
- public void testToIterableTranslationWithIsmSideInput() throws Exception {
- // A "change detector" test that makes sure the translation
- // of getting a PCollectionView<Iterable<T>> does not change
- // in bad ways during refactor
-
- DataflowPipelineOptions options = buildPipelineOptions();
- DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
-
- Pipeline pipeline = Pipeline.create(options);
- pipeline.apply(Create.of(1, 2, 3))
- .apply(View.<Integer>asIterable());
- Job job =
- translator
- .translate(
- pipeline,
- (DataflowPipelineRunner) pipeline.getRunner(),
- Collections.<DataflowPackage>emptyList())
- .getJob();
-
- List<Step> steps = job.getSteps();
- assertEquals(3, steps.size());
-
- @SuppressWarnings("unchecked")
- List<Map<String, Object>> toIsmRecordOutputs =
- (List<Map<String, Object>>) steps.get(1).getProperties().get(PropertyNames.OUTPUT_INFO);
- assertTrue(
- Structs.getBoolean(Iterables.getOnlyElement(toIsmRecordOutputs), "use_indexed_format"));
-
-
- Step collectionToSingletonStep = steps.get(2);
- assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
- }
-
- @Test
- public void testStepDisplayData() throws Exception {
- DataflowPipelineOptions options = buildPipelineOptions();
- DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
- Pipeline pipeline = Pipeline.create(options);
-
- DoFn<Integer, Integer> fn1 = new DoFn<Integer, Integer>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- c.output(c.element());
- }
-
- @Override
- public void populateDisplayData(DisplayData.Builder builder) {
- builder
- .add("foo", "bar")
- .add("foo2", DataflowPipelineTranslatorTest.class)
- .withLabel("Test Class")
- .withLinkUrl("http://www.google.com");
- }
- };
-
- DoFn<Integer, Integer> fn2 = new DoFn<Integer, Integer>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- c.output(c.element());
- }
-
- @Override
- public void populateDisplayData(DisplayData.Builder builder) {
- builder.add("foo3", 1234);
- }
- };
-
- ParDo.Bound<Integer, Integer> parDo1 = ParDo.of(fn1);
- ParDo.Bound<Integer, Integer> parDo2 = ParDo.of(fn2);
- pipeline
- .apply(Create.of(1, 2, 3))
- .apply(parDo1)
- .apply(parDo2);
-
- Job job =
- translator
- .translate(
- pipeline,
- (DataflowPipelineRunner) pipeline.getRunner(),
- Collections.<DataflowPackage>emptyList())
- .getJob();
-
- List<Step> steps = job.getSteps();
- assertEquals(3, steps.size());
-
- Map<String, Object> parDo1Properties = steps.get(1).getProperties();
- Map<String, Object> parDo2Properties = steps.get(2).getProperties();
- assertThat(parDo1Properties, hasKey("display_data"));
-
- Collection<Map<String, String>> fn1displayData =
- (Collection<Map<String, String>>) parDo1Properties.get("display_data");
- Collection<Map<String, String>> fn2displayData =
- (Collection<Map<String, String>>) parDo2Properties.get("display_data");
-
- ImmutableSet<ImmutableMap<String, Object>> expectedFn1DisplayData = ImmutableSet.of(
- ImmutableMap.<String, Object>builder()
- .put("key", "foo")
- .put("type", "STRING")
- .put("value", "bar")
- .put("namespace", fn1.getClass().getName())
- .build(),
- ImmutableMap.<String, Object>builder()
- .put("key", "fn")
- .put("type", "JAVA_CLASS")
- .put("value", fn1.getClass().getName())
- .put("shortValue", fn1.getClass().getSimpleName())
- .put("namespace", parDo1.getClass().getName())
- .build(),
- ImmutableMap.<String, Object>builder()
- .put("key", "foo2")
- .put("type", "JAVA_CLASS")
- .put("value", DataflowPipelineTranslatorTest.class.getName())
- .put("shortValue", DataflowPipelineTranslatorTest.class.getSimpleName())
- .put("namespace", fn1.getClass().getName())
- .put("label", "Test Class")
- .put("linkUrl", "http://www.google.com")
- .build()
- );
-
- ImmutableSet<ImmutableMap<String, Object>> expectedFn2DisplayData = ImmutableSet.of(
- ImmutableMap.<String, Object>builder()
- .put("key", "fn")
- .put("type", "JAVA_CLASS")
- .put("value", fn2.getClass().getName())
- .put("shortValue", fn2.getClass().getSimpleName())
- .put("namespace", parDo2.getClass().getName())
- .build(),
- ImmutableMap.<String, Object>builder()
- .put("key", "foo3")
- .put("type", "INTEGER")
- .put("value", 1234L)
- .put("namespace", fn2.getClass().getName())
- .build()
- );
-
- assertEquals(expectedFn1DisplayData, ImmutableSet.copyOf(fn1displayData));
- assertEquals(expectedFn2DisplayData, ImmutableSet.copyOf(fn2displayData));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java
deleted file mode 100644
index 2acede3..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/dataflow/CustomSourcesTest.java
+++ /dev/null
@@ -1,276 +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.runners.dataflow;
-import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.contains;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.ExpectedLogs;
-import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Sample;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.common.base.Preconditions;
-
-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.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Tests for {@link CustomSources}.
- */
-@RunWith(JUnit4.class)
-public class CustomSourcesTest {
- @Rule public ExpectedException expectedException = ExpectedException.none();
- @Rule public ExpectedLogs logged = ExpectedLogs.none(CustomSources.class);
-
- static class TestIO {
- public static Read fromRange(int from, int to) {
- return new Read(from, to, false);
- }
-
- static class Read extends BoundedSource<Integer> {
- final int from;
- final int to;
- final boolean produceTimestamps;
-
- Read(int from, int to, boolean produceTimestamps) {
- this.from = from;
- this.to = to;
- this.produceTimestamps = produceTimestamps;
- }
-
- public Read withTimestampsMillis() {
- return new Read(from, to, true);
- }
-
- @Override
- public List<Read> splitIntoBundles(long desiredBundleSizeBytes, PipelineOptions options)
- throws Exception {
- List<Read> res = new ArrayList<>();
- DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
- float step = 1.0f * (to - from) / dataflowOptions.getNumWorkers();
- for (int i = 0; i < dataflowOptions.getNumWorkers(); ++i) {
- res.add(new Read(
- Math.round(from + i * step), Math.round(from + (i + 1) * step),
- produceTimestamps));
- }
- return res;
- }
-
- @Override
- public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
- return 8 * (to - from);
- }
-
- @Override
- public boolean producesSortedKeys(PipelineOptions options) throws Exception {
- return true;
- }
-
- @Override
- public BoundedReader<Integer> createReader(PipelineOptions options) throws IOException {
- return new RangeReader(this);
- }
-
- @Override
- public void validate() {}
-
- @Override
- public String toString() {
- return "[" + from + ", " + to + ")";
- }
-
- @Override
- public Coder<Integer> getDefaultOutputCoder() {
- return BigEndianIntegerCoder.of();
- }
-
- private static class RangeReader extends BoundedReader<Integer> {
- // To verify that BasicSerializableSourceFormat calls our methods according to protocol.
- enum State {
- UNSTARTED,
- STARTED,
- FINISHED
- }
- private Read source;
- private int current = -1;
- private State state = State.UNSTARTED;
-
- public RangeReader(Read source) {
- this.source = source;
- }
-
- @Override
- public boolean start() throws IOException {
- Preconditions.checkState(state == State.UNSTARTED);
- state = State.STARTED;
- current = source.from;
- return (current < source.to);
- }
-
- @Override
- public boolean advance() throws IOException {
- Preconditions.checkState(state == State.STARTED);
- if (current == source.to - 1) {
- state = State.FINISHED;
- return false;
- }
- current++;
- return true;
- }
-
- @Override
- public Integer getCurrent() {
- Preconditions.checkState(state == State.STARTED);
- return current;
- }
-
- @Override
- public Instant getCurrentTimestamp() {
- return source.produceTimestamps
- ? new Instant(current /* as millis */) : BoundedWindow.TIMESTAMP_MIN_VALUE;
- }
-
- @Override
- public void close() throws IOException {
- Preconditions.checkState(state == State.STARTED || state == State.FINISHED);
- state = State.FINISHED;
- }
-
- @Override
- public Read getCurrentSource() {
- return source;
- }
-
- @Override
- public Read splitAtFraction(double fraction) {
- int proposedIndex = (int) (source.from + fraction * (source.to - source.from));
- if (proposedIndex <= current) {
- return null;
- }
- Read primary = new Read(source.from, proposedIndex, source.produceTimestamps);
- Read residual = new Read(proposedIndex, source.to, source.produceTimestamps);
- this.source = primary;
- return residual;
- }
-
- @Override
- public Double getFractionConsumed() {
- return (current == -1)
- ? 0.0
- : (1.0 * (1 + current - source.from) / (source.to - source.from));
- }
- }
- }
- }
-
- @Test
- public void testDirectPipelineWithoutTimestamps() throws Exception {
- Pipeline p = TestPipeline.create();
- PCollection<Integer> sum = p
- .apply(Read.from(TestIO.fromRange(10, 20)))
- .apply(Sum.integersGlobally())
- .apply(Sample.<Integer>any(1));
-
- PAssert.thatSingleton(sum).isEqualTo(145);
- p.run();
- }
-
- @Test
- public void testDirectPipelineWithTimestamps() throws Exception {
- Pipeline p = TestPipeline.create();
- PCollection<Integer> sums =
- p.apply(Read.from(TestIO.fromRange(10, 20).withTimestampsMillis()))
- .apply(Window.<Integer>into(FixedWindows.of(Duration.millis(3))))
- .apply(Sum.integersGlobally().withoutDefaults());
- // Should group into [10 11] [12 13 14] [15 16 17] [18 19].
- PAssert.that(sums).containsInAnyOrder(21, 37, 39, 48);
- p.run();
- }
-
- @Test
- public void testRangeProgressAndSplitAtFraction() throws Exception {
- // Show basic usage of getFractionConsumed and splitAtFraction.
- // This test only tests TestIO itself, not BasicSerializableSourceFormat.
-
- DataflowPipelineOptions options =
- PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
- TestIO.Read source = TestIO.fromRange(10, 20);
- try (BoundedSource.BoundedReader<Integer> reader = source.createReader(options)) {
- assertEquals(0, reader.getFractionConsumed().intValue());
- assertTrue(reader.start());
- assertEquals(0.1, reader.getFractionConsumed(), 1e-6);
- assertTrue(reader.advance());
- assertEquals(0.2, reader.getFractionConsumed(), 1e-6);
- // Already past 0.0 and 0.1.
- assertNull(reader.splitAtFraction(0.0));
- assertNull(reader.splitAtFraction(0.1));
-
- {
- TestIO.Read residual = (TestIO.Read) reader.splitAtFraction(0.5);
- assertNotNull(residual);
- TestIO.Read primary = (TestIO.Read) reader.getCurrentSource();
- assertThat(readFromSource(primary, options), contains(10, 11, 12, 13, 14));
- assertThat(readFromSource(residual, options), contains(15, 16, 17, 18, 19));
- }
-
- // Range is now [10, 15) and we are at 12.
- {
- TestIO.Read residual = (TestIO.Read) reader.splitAtFraction(0.8); // give up 14.
- assertNotNull(residual);
- TestIO.Read primary = (TestIO.Read) reader.getCurrentSource();
- assertThat(readFromSource(primary, options), contains(10, 11, 12, 13));
- assertThat(readFromSource(residual, options), contains(14));
- }
-
- assertTrue(reader.advance());
- assertEquals(12, reader.getCurrent().intValue());
- assertTrue(reader.advance());
- assertEquals(13, reader.getCurrent().intValue());
- assertFalse(reader.advance());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java
deleted file mode 100644
index 185ab51..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunnerTest.java
+++ /dev/null
@@ -1,379 +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.testing;
-
-import static org.hamcrest.Matchers.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineJob;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.util.GcsUtil;
-import org.apache.beam.sdk.util.MonitoringUtil;
-import org.apache.beam.sdk.util.MonitoringUtil.JobMessagesHandler;
-import org.apache.beam.sdk.util.NoopPathValidator;
-import org.apache.beam.sdk.util.TestCredential;
-import org.apache.beam.sdk.util.TimeUtil;
-import org.apache.beam.sdk.util.Transport;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.api.client.http.LowLevelHttpResponse;
-import com.google.api.client.json.Json;
-import com.google.api.client.testing.http.MockHttpTransport;
-import com.google.api.client.testing.http.MockLowLevelHttpRequest;
-import com.google.api.client.testing.http.MockLowLevelHttpResponse;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.JobMetrics;
-import com.google.api.services.dataflow.model.MetricStructuredName;
-import com.google.api.services.dataflow.model.MetricUpdate;
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.MockitoAnnotations;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.concurrent.TimeUnit;
-
-/** Tests for {@link TestDataflowPipelineRunner}. */
-@RunWith(JUnit4.class)
-public class TestDataflowPipelineRunnerTest {
- @Rule public ExpectedException expectedException = ExpectedException.none();
- @Mock private MockHttpTransport transport;
- @Mock private MockLowLevelHttpRequest request;
- @Mock private GcsUtil mockGcsUtil;
-
- private TestDataflowPipelineOptions options;
- private Dataflow service;
-
- @Before
- public void setUp() throws Exception {
- MockitoAnnotations.initMocks(this);
- when(transport.buildRequest(anyString(), anyString())).thenReturn(request);
- doCallRealMethod().when(request).getContentAsString();
- service = new Dataflow(transport, Transport.getJsonFactory(), null);
-
- options = PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
- options.setAppName("TestAppName");
- options.setProject("test-project");
- options.setTempLocation("gs://test/temp/location");
- options.setTempRoot("gs://test");
- options.setGcpCredential(new TestCredential());
- options.setDataflowClient(service);
- options.setRunner(TestDataflowPipelineRunner.class);
- options.setPathValidatorClass(NoopPathValidator.class);
- }
-
- @Test
- public void testToString() {
- assertEquals("TestDataflowPipelineRunner#TestAppName",
- new TestDataflowPipelineRunner(options).toString());
- }
-
- @Test
- public void testRunBatchJobThatSucceeds() throws Exception {
- Pipeline p = TestPipeline.create(options);
- PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
- PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
- 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();
- when(request.execute()).thenReturn(
- generateMockMetricResponse(true /* success */, true /* tentative */));
- assertEquals(mockJob, runner.run(p, mockRunner));
- }
-
- @Test
- public void testRunBatchJobThatFails() throws Exception {
- Pipeline p = TestPipeline.create(options);
- PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
- PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
- 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();
- try {
- runner.run(p, mockRunner);
- } catch (AssertionError expected) {
- return;
- }
- // Note that fail throws an AssertionError which is why it is placed out here
- // instead of inside the try-catch block.
- fail("AssertionError expected");
- }
-
- @Test
- public void testBatchPipelineFailsIfException() throws Exception {
- Pipeline p = TestPipeline.create(options);
- PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
- PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
- DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
- when(mockJob.getDataflowClient()).thenReturn(service);
- when(mockJob.getState()).thenReturn(State.RUNNING);
- when(mockJob.getProjectId()).thenReturn("test-project");
- when(mockJob.getJobId()).thenReturn("test-job");
- when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
- .thenAnswer(new Answer<State>() {
- @Override
- public State answer(InvocationOnMock invocation) {
- JobMessage message = new JobMessage();
- message.setMessageText("FooException");
- message.setTime(TimeUtil.toCloudTime(Instant.now()));
- message.setMessageImportance("JOB_MESSAGE_ERROR");
- ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
- .process(Arrays.asList(message));
- return State.CANCELLED;
- }
- });
-
- DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
- when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
- when(request.execute()).thenReturn(
- generateMockMetricResponse(false /* success */, true /* tentative */));
- TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
- try {
- runner.run(p, mockRunner);
- } catch (AssertionError expected) {
- assertThat(expected.getMessage(), containsString("FooException"));
- verify(mockJob, atLeastOnce()).cancel();
- return;
- }
- // Note that fail throws an AssertionError which is why it is placed out here
- // instead of inside the try-catch block.
- fail("AssertionError expected");
- }
-
- @Test
- public void testRunStreamingJobThatSucceeds() throws Exception {
- options.setStreaming(true);
- Pipeline p = TestPipeline.create(options);
- PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
- PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
- DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
- when(mockJob.getDataflowClient()).thenReturn(service);
- when(mockJob.getState()).thenReturn(State.RUNNING);
- 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);
-
- when(request.execute()).thenReturn(
- generateMockMetricResponse(true /* success */, true /* tentative */));
- TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
- runner.run(p, mockRunner);
- }
-
- @Test
- public void testRunStreamingJobThatFails() throws Exception {
- options.setStreaming(true);
- Pipeline p = TestPipeline.create(options);
- PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
- PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
- DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
- when(mockJob.getDataflowClient()).thenReturn(service);
- when(mockJob.getState()).thenReturn(State.RUNNING);
- 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);
-
- when(request.execute()).thenReturn(
- generateMockMetricResponse(false /* success */, true /* tentative */));
- TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
- try {
- runner.run(p, mockRunner);
- } catch (AssertionError expected) {
- return;
- }
- // Note that fail throws an AssertionError which is why it is placed out here
- // instead of inside the try-catch block.
- fail("AssertionError expected");
- }
-
- @Test
- public void testCheckingForSuccessWhenPAssertSucceeds() throws Exception {
- DataflowPipelineJob job =
- spy(new DataflowPipelineJob("test-project", "test-job", service, null));
- Pipeline p = TestPipeline.create(options);
- PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
- PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
- TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
- when(request.execute()).thenReturn(
- generateMockMetricResponse(true /* success */, true /* tentative */));
- doReturn(State.DONE).when(job).getState();
- assertEquals(Optional.of(true), runner.checkForSuccess(job));
- }
-
- @Test
- public void testCheckingForSuccessWhenPAssertFails() throws Exception {
- DataflowPipelineJob job =
- spy(new DataflowPipelineJob("test-project", "test-job", service, null));
- Pipeline p = TestPipeline.create(options);
- PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
- PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
- TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
- when(request.execute()).thenReturn(
- generateMockMetricResponse(false /* success */, true /* tentative */));
- doReturn(State.DONE).when(job).getState();
- assertEquals(Optional.of(false), runner.checkForSuccess(job));
- }
-
- @Test
- public void testCheckingForSuccessSkipsNonTentativeMetrics() throws Exception {
- DataflowPipelineJob job =
- spy(new DataflowPipelineJob("test-project", "test-job", service, null));
- Pipeline p = TestPipeline.create(options);
- PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
- PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
- TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
- when(request.execute()).thenReturn(
- generateMockMetricResponse(true /* success */, false /* tentative */));
- doReturn(State.RUNNING).when(job).getState();
- assertEquals(Optional.absent(), runner.checkForSuccess(job));
- }
-
- private LowLevelHttpResponse generateMockMetricResponse(boolean success, boolean tentative)
- throws Exception {
- MetricStructuredName name = new MetricStructuredName();
- name.setName(success ? "PAssertSuccess" : "PAssertFailure");
- name.setContext(
- tentative ? ImmutableMap.of("tentative", "") : ImmutableMap.<String, String>of());
-
- MetricUpdate metric = new MetricUpdate();
- metric.setName(name);
- metric.setScalar(BigDecimal.ONE);
-
- MockLowLevelHttpResponse response = new MockLowLevelHttpResponse();
- response.setContentType(Json.MEDIA_TYPE);
- JobMetrics jobMetrics = new JobMetrics();
- jobMetrics.setMetrics(Lists.newArrayList(metric));
- // N.B. Setting the factory is necessary in order to get valid JSON.
- jobMetrics.setFactory(Transport.getJsonFactory());
- response.setContent(jobMetrics.toPrettyString());
- return response;
- }
-
- @Test
- public void testStreamingPipelineFailsIfServiceFails() throws Exception {
- DataflowPipelineJob job =
- spy(new DataflowPipelineJob("test-project", "test-job", service, null));
- Pipeline p = TestPipeline.create(options);
- PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
- PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
- TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
- when(request.execute()).thenReturn(
- generateMockMetricResponse(true /* success */, false /* tentative */));
- doReturn(State.FAILED).when(job).getState();
- assertEquals(Optional.of(false), runner.checkForSuccess(job));
- }
-
- @Test
- public void testStreamingPipelineFailsIfException() throws Exception {
- options.setStreaming(true);
- Pipeline p = TestPipeline.create(options);
- PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
- PAssert.that(pc).containsInAnyOrder(1, 2, 3);
-
- DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
- when(mockJob.getDataflowClient()).thenReturn(service);
- when(mockJob.getState()).thenReturn(State.RUNNING);
- when(mockJob.getProjectId()).thenReturn("test-project");
- when(mockJob.getJobId()).thenReturn("test-job");
- when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
- .thenAnswer(new Answer<State>() {
- @Override
- public State answer(InvocationOnMock invocation) {
- JobMessage message = new JobMessage();
- message.setMessageText("FooException");
- message.setTime(TimeUtil.toCloudTime(Instant.now()));
- message.setMessageImportance("JOB_MESSAGE_ERROR");
- ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
- .process(Arrays.asList(message));
- return State.CANCELLED;
- }
- });
-
- DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
- when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
-
- when(request.execute()).thenReturn(
- generateMockMetricResponse(false /* success */, true /* tentative */));
- TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
- try {
- runner.run(p, mockRunner);
- } catch (AssertionError expected) {
- assertThat(expected.getMessage(), containsString("FooException"));
- verify(mockJob, atLeastOnce()).cancel();
- return;
- }
- // Note that fail throws an AssertionError which is why it is placed out here
- // instead of inside the try-catch block.
- fail("AssertionError expected");
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java
deleted file mode 100644
index a381f68..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowGroupByKeyTest.java
+++ /dev/null
@@ -1,110 +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.transforms;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
-import org.apache.beam.sdk.transforms.windowing.Sessions;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.NoopPathValidator;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.TypeDescriptor;
-
-import org.joda.time.Duration;
-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.Arrays;
-import java.util.List;
-
-/** Tests for {@link GroupByKey} for the {@link DataflowPipelineRunner}. */
-@RunWith(JUnit4.class)
-public class DataflowGroupByKeyTest {
- @Rule
- public ExpectedException thrown = ExpectedException.none();
-
- /**
- * Create a test pipeline that uses the {@link DataflowPipelineRunner} so that {@link GroupByKey}
- * is not expanded. This is used for verifying that even without expansion the proper errors show
- * up.
- */
- private Pipeline createTestServiceRunner() {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setProject("someproject");
- options.setStagingLocation("gs://staging");
- options.setPathValidatorClass(NoopPathValidator.class);
- options.setDataflowClient(null);
- return Pipeline.create(options);
- }
-
- @Test
- public void testInvalidWindowsService() {
- Pipeline p = createTestServiceRunner();
-
- List<KV<String, Integer>> ungroupedPairs = Arrays.asList();
-
- PCollection<KV<String, Integer>> input =
- p.apply(Create.of(ungroupedPairs)
- .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())))
- .apply(Window.<KV<String, Integer>>into(
- Sessions.withGapDuration(Duration.standardMinutes(1))));
-
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("GroupByKey must have a valid Window merge function");
- input
- .apply("GroupByKey", GroupByKey.<String, Integer>create())
- .apply("GroupByKeyAgain", GroupByKey.<String, Iterable<Integer>>create());
- }
-
- @Test
- public void testGroupByKeyServiceUnbounded() {
- Pipeline p = createTestServiceRunner();
-
- PCollection<KV<String, Integer>> input =
- p.apply(
- new PTransform<PBegin, PCollection<KV<String, Integer>>>() {
- @Override
- public PCollection<KV<String, Integer>> apply(PBegin input) {
- return PCollection.<KV<String, Integer>>createPrimitiveOutputInternal(
- input.getPipeline(),
- WindowingStrategy.globalDefault(),
- PCollection.IsBounded.UNBOUNDED)
- .setTypeDescriptorInternal(new TypeDescriptor<KV<String, Integer>>() {});
- }
- });
-
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage(
- "GroupByKey cannot be applied to non-bounded PCollection in the GlobalWindow without "
- + "a trigger. Use a Window.into or Window.triggering transform prior to GroupByKey.");
-
- input.apply("GroupByKey", GroupByKey.<String, Integer>create());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java
deleted file mode 100644
index b86de7e..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/transforms/DataflowViewTest.java
+++ /dev/null
@@ -1,205 +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.transforms;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.InvalidWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.NoopPathValidator;
-import org.apache.beam.sdk.util.WindowingStrategy;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PBegin;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.TypeDescriptor;
-
-import org.hamcrest.Matchers;
-import org.joda.time.Duration;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.internal.matchers.ThrowableMessageMatcher;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link View} for a {@link DataflowPipelineRunner}. */
-@RunWith(JUnit4.class)
-public class DataflowViewTest {
- @Rule
- public transient ExpectedException thrown = ExpectedException.none();
-
- private Pipeline createTestBatchRunner() {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setProject("someproject");
- options.setStagingLocation("gs://staging");
- options.setPathValidatorClass(NoopPathValidator.class);
- options.setDataflowClient(null);
- return Pipeline.create(options);
- }
-
- private Pipeline createTestStreamingRunner() {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setRunner(DataflowPipelineRunner.class);
- options.setStreaming(true);
- options.setProject("someproject");
- options.setStagingLocation("gs://staging");
- options.setPathValidatorClass(NoopPathValidator.class);
- options.setDataflowClient(null);
- return Pipeline.create(options);
- }
-
- private void testViewUnbounded(
- Pipeline pipeline,
- PTransform<PCollection<KV<String, Integer>>, ? extends PCollectionView<?>> view) {
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("Unable to create a side-input view from input");
- thrown.expectCause(
- ThrowableMessageMatcher.hasMessage(Matchers.containsString("non-bounded PCollection")));
- pipeline
- .apply(
- new PTransform<PBegin, PCollection<KV<String, Integer>>>() {
- @Override
- public PCollection<KV<String, Integer>> apply(PBegin input) {
- return PCollection.<KV<String, Integer>>createPrimitiveOutputInternal(
- input.getPipeline(),
- WindowingStrategy.globalDefault(),
- PCollection.IsBounded.UNBOUNDED)
- .setTypeDescriptorInternal(new TypeDescriptor<KV<String, Integer>>() {});
- }
- })
- .apply(view);
- }
-
- private void testViewNonmerging(
- Pipeline pipeline,
- PTransform<PCollection<KV<String, Integer>>, ? extends PCollectionView<?>> view) {
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage("Unable to create a side-input view from input");
- thrown.expectCause(
- ThrowableMessageMatcher.hasMessage(Matchers.containsString("Consumed by GroupByKey")));
- pipeline.apply(Create.<KV<String, Integer>>of(KV.of("hello", 5)))
- .apply(Window.<KV<String, Integer>>into(new InvalidWindows<>(
- "Consumed by GroupByKey", FixedWindows.of(Duration.standardHours(1)))))
- .apply(view);
- }
-
- @Test
- public void testViewUnboundedAsSingletonBatch() {
- testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asSingleton());
- }
-
- @Test
- public void testViewUnboundedAsSingletonStreaming() {
- testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asSingleton());
- }
-
- @Test
- public void testViewUnboundedAsIterableBatch() {
- testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asIterable());
- }
-
- @Test
- public void testViewUnboundedAsIterableStreaming() {
- testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asIterable());
- }
-
- @Test
- public void testViewUnboundedAsListBatch() {
- testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asList());
- }
-
- @Test
- public void testViewUnboundedAsListStreaming() {
- testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asList());
- }
-
- @Test
- public void testViewUnboundedAsMapBatch() {
- testViewUnbounded(createTestBatchRunner(), View.<String, Integer>asMap());
- }
-
- @Test
- public void testViewUnboundedAsMapStreaming() {
- testViewUnbounded(createTestStreamingRunner(), View.<String, Integer>asMap());
- }
-
- @Test
- public void testViewUnboundedAsMultimapBatch() {
- testViewUnbounded(createTestBatchRunner(), View.<String, Integer>asMultimap());
- }
-
- @Test
- public void testViewUnboundedAsMultimapStreaming() {
- testViewUnbounded(createTestStreamingRunner(), View.<String, Integer>asMultimap());
- }
-
- @Test
- public void testViewNonmergingAsSingletonBatch() {
- testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asSingleton());
- }
-
- @Test
- public void testViewNonmergingAsSingletonStreaming() {
- testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asSingleton());
- }
-
- @Test
- public void testViewNonmergingAsIterableBatch() {
- testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asIterable());
- }
-
- @Test
- public void testViewNonmergingAsIterableStreaming() {
- testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asIterable());
- }
-
- @Test
- public void testViewNonmergingAsListBatch() {
- testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asList());
- }
-
- @Test
- public void testViewNonmergingAsListStreaming() {
- testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asList());
- }
-
- @Test
- public void testViewNonmergingAsMapBatch() {
- testViewNonmerging(createTestBatchRunner(), View.<String, Integer>asMap());
- }
-
- @Test
- public void testViewNonmergingAsMapStreaming() {
- testViewNonmerging(createTestStreamingRunner(), View.<String, Integer>asMap());
- }
-
- @Test
- public void testViewNonmergingAsMultimapBatch() {
- testViewNonmerging(createTestBatchRunner(), View.<String, Integer>asMultimap());
- }
-
- @Test
- public void testViewNonmergingAsMultimapStreaming() {
- testViewNonmerging(createTestStreamingRunner(), View.<String, Integer>asMultimap());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java
deleted file mode 100644
index b459c47..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/DataflowPathValidatorTest.java
+++ /dev/null
@@ -1,92 +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.util;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-/** Tests for {@link DataflowPathValidator}. */
-@RunWith(JUnit4.class)
-public class DataflowPathValidatorTest {
- @Rule public ExpectedException expectedException = ExpectedException.none();
-
- @Mock private GcsUtil mockGcsUtil;
- private DataflowPathValidator validator;
-
- @Before
- public void setUp() throws Exception {
- MockitoAnnotations.initMocks(this);
- when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(true);
- when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setGcpCredential(new TestCredential());
- options.setRunner(DataflowPipelineRunner.class);
- options.setGcsUtil(mockGcsUtil);
- validator = new DataflowPathValidator(options);
- }
-
- @Test
- public void testValidFilePattern() {
- validator.validateInputFilePatternSupported("gs://bucket/path");
- }
-
- @Test
- public void testInvalidFilePattern() {
- expectedException.expect(IllegalArgumentException.class);
- expectedException.expectMessage(
- "DataflowPipelineRunner expected a valid 'gs://' path but was given '/local/path'");
- validator.validateInputFilePatternSupported("/local/path");
- }
-
- @Test
- public void testWhenBucketDoesNotExist() throws Exception {
- when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(false);
- expectedException.expect(IllegalArgumentException.class);
- expectedException.expectMessage(
- "Could not find file gs://non-existent-bucket/location");
- validator.validateInputFilePatternSupported("gs://non-existent-bucket/location");
- }
-
- @Test
- public void testValidOutputPrefix() {
- validator.validateOutputFilePrefixSupported("gs://bucket/path");
- }
-
- @Test
- public void testInvalidOutputPrefix() {
- expectedException.expect(IllegalArgumentException.class);
- expectedException.expectMessage(
- "DataflowPipelineRunner expected a valid 'gs://' path but was given '/local/path'");
- validator.validateOutputFilePrefixSupported("/local/path");
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java
deleted file mode 100644
index bdc0bc3..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/MonitoringUtilTest.java
+++ /dev/null
@@ -1,149 +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.util;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.ListJobMessagesResponse;
-
-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.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Tests for MonitoringUtil.
- */
-@RunWith(JUnit4.class)
-public class MonitoringUtilTest {
- private static final String PROJECT_ID = "someProject";
- private static final String JOB_ID = "1234";
-
- @Rule public ExpectedException thrown = ExpectedException.none();
-
- @Test
- public void testGetJobMessages() throws IOException {
- Dataflow.Projects.Jobs.Messages mockMessages = mock(Dataflow.Projects.Jobs.Messages.class);
-
- // Two requests are needed to get all the messages.
- Dataflow.Projects.Jobs.Messages.List firstRequest =
- mock(Dataflow.Projects.Jobs.Messages.List.class);
- Dataflow.Projects.Jobs.Messages.List secondRequest =
- mock(Dataflow.Projects.Jobs.Messages.List.class);
-
- when(mockMessages.list(PROJECT_ID, JOB_ID)).thenReturn(firstRequest).thenReturn(secondRequest);
-
- ListJobMessagesResponse firstResponse = new ListJobMessagesResponse();
- firstResponse.setJobMessages(new ArrayList<JobMessage>());
- for (int i = 0; i < 100; ++i) {
- JobMessage message = new JobMessage();
- message.setId("message_" + i);
- message.setTime(TimeUtil.toCloudTime(new Instant(i)));
- firstResponse.getJobMessages().add(message);
- }
- String pageToken = "page_token";
- firstResponse.setNextPageToken(pageToken);
-
- ListJobMessagesResponse secondResponse = new ListJobMessagesResponse();
- secondResponse.setJobMessages(new ArrayList<JobMessage>());
- for (int i = 100; i < 150; ++i) {
- JobMessage message = new JobMessage();
- message.setId("message_" + i);
- message.setTime(TimeUtil.toCloudTime(new Instant(i)));
- secondResponse.getJobMessages().add(message);
- }
-
- when(firstRequest.execute()).thenReturn(firstResponse);
- when(secondRequest.execute()).thenReturn(secondResponse);
-
- MonitoringUtil util = new MonitoringUtil(PROJECT_ID, mockMessages);
-
- List<JobMessage> messages = util.getJobMessages(JOB_ID, -1);
-
- verify(secondRequest).setPageToken(pageToken);
-
- assertEquals(150, messages.size());
- }
-
- @Test
- public void testToStateCreatesState() {
- String stateName = "JOB_STATE_DONE";
-
- State result = MonitoringUtil.toState(stateName);
-
- assertEquals(State.DONE, result);
- }
-
- @Test
- public void testToStateWithNullReturnsUnknown() {
- String stateName = null;
-
- State result = MonitoringUtil.toState(stateName);
-
- assertEquals(State.UNKNOWN, result);
- }
-
- @Test
- public void testToStateWithOtherValueReturnsUnknown() {
- String stateName = "FOO_BAR_BAZ";
-
- State result = MonitoringUtil.toState(stateName);
-
- assertEquals(State.UNKNOWN, result);
- }
-
- @Test
- public void testDontOverrideEndpointWithDefaultApi() {
- DataflowPipelineOptions options =
- PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
- options.setProject(PROJECT_ID);
- options.setGcpCredential(new TestCredential());
- String cancelCommand = MonitoringUtil.getGcloudCancelCommand(options, JOB_ID);
- assertEquals("gcloud alpha dataflow jobs --project=someProject cancel 1234", cancelCommand);
- }
-
- @Test
- public void testOverridesEndpointWithStagedDataflowEndpoint() {
- DataflowPipelineOptions options =
- PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
- options.setProject(PROJECT_ID);
- options.setGcpCredential(new TestCredential());
- String stagingDataflowEndpoint = "v0neverExisted";
- options.setDataflowEndpoint(stagingDataflowEndpoint);
- String cancelCommand = MonitoringUtil.getGcloudCancelCommand(options, JOB_ID);
- assertEquals(
- "CLOUDSDK_API_ENDPOINT_OVERRIDES_DATAFLOW=https://dataflow.googleapis.com/v0neverExisted/ "
- + "gcloud alpha dataflow jobs --project=someProject cancel 1234",
- cancelCommand);
- }
-}
[21/21] incubator-beam git commit: Closes #239
Posted by dh...@apache.org.
Closes #239
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/e3105c8e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/e3105c8e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/e3105c8e
Branch: refs/heads/master
Commit: e3105c8e109535f801fd145b91b0c7aa93b86d1a
Parents: 5e3d7ad 0fafd4e
Author: Dan Halperin <dh...@google.com>
Authored: Tue Apr 26 18:07:43 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Tue Apr 26 18:07:43 2016 -0700
----------------------------------------------------------------------
.../apache/beam/examples/MinimalWordCount.java | 4 +-
.../org/apache/beam/examples/WordCount.java | 2 +-
.../examples/common/DataflowExampleOptions.java | 2 +-
.../examples/common/DataflowExampleUtils.java | 8 +-
.../common/ExampleBigQueryTableOptions.java | 2 +-
...xamplePubsubTopicAndSubscriptionOptions.java | 2 +-
.../common/ExamplePubsubTopicOptions.java | 2 +-
.../examples/common/PubsubFileInjector.java | 2 +-
.../beam/examples/complete/AutoComplete.java | 2 +-
.../examples/complete/StreamingWordExtract.java | 2 +-
.../examples/complete/TopWikipediaSessions.java | 2 +-
.../beam/examples/cookbook/DeDupExample.java | 2 +-
.../beam/examples/cookbook/TriggerExample.java | 4 +-
.../org/apache/beam/examples/WordCountIT.java | 4 +-
.../beam/examples/MinimalWordCountJava8.java | 4 +-
.../beam/examples/complete/game/GameStats.java | 14 +-
.../examples/complete/game/HourlyTeamScore.java | 6 +-
.../examples/complete/game/LeaderBoard.java | 8 +-
.../beam/runners/flink/examples/TFIDF.java | 1 +
.../beam/runners/flink/examples/WordCount.java | 9 +-
.../flink/examples/streaming/AutoComplete.java | 21 +-
.../flink/examples/streaming/JoinExamples.java | 6 +-
.../KafkaWindowedWordCountExample.java | 11 +-
.../examples/streaming/WindowedWordCount.java | 13 +-
.../runners/flink/FlinkPipelineOptions.java | 2 +-
.../beam/runners/flink/FlinkPipelineRunner.java | 4 +-
.../FlinkBatchPipelineTranslator.java | 2 +-
.../FlinkStreamingPipelineTranslator.java | 2 +-
runners/google-cloud-dataflow-java/pom.xml | 4 +-
.../BlockingDataflowPipelineRunner.java | 186 ++
.../DataflowJobAlreadyExistsException.java | 35 +
.../DataflowJobAlreadyUpdatedException.java | 34 +
.../dataflow/DataflowJobCancelledException.java | 39 +
.../runners/dataflow/DataflowJobException.java | 41 +
.../dataflow/DataflowJobExecutionException.java | 35 +
.../dataflow/DataflowJobUpdatedException.java | 51 +
.../runners/dataflow/DataflowPipelineJob.java | 397 +++
.../dataflow/DataflowPipelineRegistrar.java | 62 +
.../dataflow/DataflowPipelineRunner.java | 3025 ++++++++++++++++++
.../dataflow/DataflowPipelineRunnerHooks.java | 39 +
.../dataflow/DataflowPipelineTranslator.java | 1059 ++++++
.../dataflow/DataflowServiceException.java | 33 +
.../dataflow/internal/AssignWindows.java | 89 +
.../dataflow/internal/BigQueryIOTranslator.java | 72 +
.../dataflow/internal/CustomSources.java | 121 +
.../internal/DataflowAggregatorTransforms.java | 81 +
.../internal/DataflowMetricUpdateExtractor.java | 111 +
.../dataflow/internal/PubsubIOTranslator.java | 108 +
.../dataflow/internal/ReadTranslator.java | 105 +
.../runners/dataflow/internal/package-info.java | 21 +
.../BlockingDataflowPipelineOptions.java | 55 +
.../dataflow/options/CloudDebuggerOptions.java | 56 +
.../options/DataflowPipelineDebugOptions.java | 247 ++
.../options/DataflowPipelineOptions.java | 126 +
.../DataflowPipelineWorkerPoolOptions.java | 263 ++
.../options/DataflowProfilingOptions.java | 50 +
.../options/DataflowWorkerHarnessOptions.java | 55 +
.../options/DataflowWorkerLoggingOptions.java | 159 +
.../testing/TestDataflowPipelineOptions.java | 30 +
.../testing/TestDataflowPipelineRunner.java | 273 ++
.../dataflow/util/DataflowPathValidator.java | 100 +
.../dataflow/util/DataflowTransport.java | 114 +
.../beam/runners/dataflow/util/GcsStager.java | 55 +
.../runners/dataflow/util/MonitoringUtil.java | 237 ++
.../beam/runners/dataflow/util/PackageUtil.java | 333 ++
.../beam/runners/dataflow/util/Stager.java | 30 +
.../BlockingDataflowPipelineOptions.java | 50 -
.../beam/sdk/options/CloudDebuggerOptions.java | 53 -
.../options/DataflowPipelineDebugOptions.java | 242 --
.../sdk/options/DataflowPipelineOptions.java | 115 -
.../DataflowPipelineWorkerPoolOptions.java | 258 --
.../sdk/options/DataflowProfilingOptions.java | 48 -
.../options/DataflowWorkerHarnessOptions.java | 51 -
.../options/DataflowWorkerLoggingOptions.java | 155 -
.../runners/BlockingDataflowPipelineRunner.java | 185 --
.../DataflowJobAlreadyExistsException.java | 35 -
.../DataflowJobAlreadyUpdatedException.java | 34 -
.../runners/DataflowJobCancelledException.java | 39 -
.../beam/sdk/runners/DataflowJobException.java | 41 -
.../runners/DataflowJobExecutionException.java | 35 -
.../runners/DataflowJobUpdatedException.java | 51 -
.../beam/sdk/runners/DataflowPipelineJob.java | 395 ---
.../sdk/runners/DataflowPipelineRegistrar.java | 60 -
.../sdk/runners/DataflowPipelineRunner.java | 3022 -----------------
.../runners/DataflowPipelineRunnerHooks.java | 39 -
.../sdk/runners/DataflowPipelineTranslator.java | 1058 ------
.../sdk/runners/DataflowServiceException.java | 33 -
.../sdk/runners/dataflow/AssignWindows.java | 89 -
.../runners/dataflow/BigQueryIOTranslator.java | 72 -
.../sdk/runners/dataflow/CustomSources.java | 121 -
.../dataflow/DataflowAggregatorTransforms.java | 81 -
.../dataflow/DataflowMetricUpdateExtractor.java | 111 -
.../runners/dataflow/PubsubIOTranslator.java | 108 -
.../sdk/runners/dataflow/ReadTranslator.java | 105 -
.../beam/sdk/runners/dataflow/package-info.java | 21 -
.../testing/TestDataflowPipelineOptions.java | 28 -
.../sdk/testing/TestDataflowPipelineRunner.java | 271 --
.../beam/sdk/util/DataflowPathValidator.java | 99 -
.../apache/beam/sdk/util/DataflowTransport.java | 113 -
.../org/apache/beam/sdk/util/GcsStager.java | 55 -
.../apache/beam/sdk/util/MonitoringUtil.java | 236 --
.../org/apache/beam/sdk/util/PackageUtil.java | 328 --
.../java/org/apache/beam/sdk/util/Stager.java | 30 -
.../BlockingDataflowPipelineRunnerTest.java | 302 ++
.../dataflow/DataflowPipelineJobTest.java | 608 ++++
.../dataflow/DataflowPipelineRegistrarTest.java | 75 +
.../dataflow/DataflowPipelineRunnerTest.java | 1401 ++++++++
.../DataflowPipelineTranslatorTest.java | 967 ++++++
.../dataflow/internal/CustomSourcesTest.java | 276 ++
.../runners/dataflow/io/DataflowTextIOTest.java | 119 +
.../DataflowPipelineDebugOptionsTest.java | 43 +
.../options/DataflowPipelineOptionsTest.java | 93 +
.../options/DataflowProfilingOptionsTest.java | 51 +
.../DataflowWorkerLoggingOptionsTest.java | 77 +
.../testing/TestDataflowPipelineRunnerTest.java | 381 +++
.../transforms/DataflowGroupByKeyTest.java | 113 +
.../dataflow/transforms/DataflowViewTest.java | 208 ++
.../util/DataflowPathValidatorTest.java | 94 +
.../dataflow/util/MonitoringUtilTest.java | 151 +
.../runners/dataflow/util/PackageUtilTest.java | 486 +++
.../apache/beam/sdk/io/DataflowTextIOTest.java | 118 -
.../DataflowPipelineDebugOptionsTest.java | 41 -
.../options/DataflowPipelineOptionsTest.java | 92 -
.../options/DataflowProfilingOptionsTest.java | 49 -
.../DataflowWorkerLoggingOptionsTest.java | 77 -
.../BlockingDataflowPipelineRunnerTest.java | 302 --
.../sdk/runners/DataflowPipelineJobTest.java | 606 ----
.../runners/DataflowPipelineRegistrarTest.java | 74 -
.../sdk/runners/DataflowPipelineRunnerTest.java | 1400 --------
.../runners/DataflowPipelineTranslatorTest.java | 965 ------
.../sdk/runners/dataflow/CustomSourcesTest.java | 276 --
.../testing/TestDataflowPipelineRunnerTest.java | 379 ---
.../sdk/transforms/DataflowGroupByKeyTest.java | 110 -
.../beam/sdk/transforms/DataflowViewTest.java | 205 --
.../sdk/util/DataflowPathValidatorTest.java | 92 -
.../beam/sdk/util/MonitoringUtilTest.java | 149 -
.../apache/beam/sdk/util/PackageUtilTest.java | 484 ---
.../org/apache/beam/sdk/testing/PAssert.java | 8 +-
.../apache/beam/sdk/testing/TestPipeline.java | 2 +-
.../java/org/apache/beam/sdk/util/ZipFiles.java | 2 +-
sdks/java/java8tests/pom.xml | 3 +
.../src/main/java/MinimalWordCount.java | 4 +-
.../src/main/java/WindowedWordCount.java | 10 +-
.../src/main/java/WordCount.java | 2 +-
.../java/common/DataflowExampleOptions.java | 2 +-
.../main/java/common/DataflowExampleUtils.java | 9 +-
.../common/ExampleBigQueryTableOptions.java | 5 +-
.../java/common/ExamplePubsubTopicOptions.java | 2 +-
.../main/java/common/PubsubFileInjector.java | 9 +-
149 files changed, 13503 insertions(+), 13360 deletions(-)
----------------------------------------------------------------------
[20/21] incubator-beam git commit: Fix a few underlying checkstyle
issues in java8 examples
Posted by dh...@apache.org.
Fix a few underlying checkstyle issues in java8 examples
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/9e19efdf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/9e19efdf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/9e19efdf
Branch: refs/heads/master
Commit: 9e19efdf37541ee082bed1ebfd6dd0b154de5f0a
Parents: c08f973
Author: Davor Bonaci <da...@google.com>
Authored: Mon Apr 25 15:02:27 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Apr 26 17:59:39 2016 -0700
----------------------------------------------------------------------
.../apache/beam/examples/complete/game/GameStats.java | 12 ++++++++----
.../beam/examples/complete/game/HourlyTeamScore.java | 6 ++++--
.../apache/beam/examples/complete/game/LeaderBoard.java | 6 ++++--
3 files changed, 16 insertions(+), 8 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9e19efdf/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
----------------------------------------------------------------------
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 d93c2ae..2d14264 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
@@ -207,8 +207,10 @@ public class GameStats extends LeaderBoard {
c -> c.element().getValue()));
tableConfigure.put("window_start",
new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
- c -> { IntervalWindow w = (IntervalWindow) c.window();
- return fmt.print(w.start()); }));
+ c -> {
+ IntervalWindow w = (IntervalWindow) c.window();
+ return fmt.print(w.start());
+ }));
tableConfigure.put("processing_time",
new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
"STRING", c -> fmt.print(Instant.now())));
@@ -226,8 +228,10 @@ public class GameStats extends LeaderBoard {
new HashMap<String, WriteWindowedToBigQuery.FieldInfo<Double>>();
tableConfigure.put("window_start",
new WriteWindowedToBigQuery.FieldInfo<Double>("STRING",
- c -> { IntervalWindow w = (IntervalWindow) c.window();
- return fmt.print(w.start()); }));
+ c -> {
+ IntervalWindow w = (IntervalWindow) c.window();
+ return fmt.print(w.start());
+ }));
tableConfigure.put("mean_duration",
new WriteWindowedToBigQuery.FieldInfo<Double>("FLOAT", c -> c.element()));
return tableConfigure;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9e19efdf/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
----------------------------------------------------------------------
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 5ce7d95..b516a32 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
@@ -132,8 +132,10 @@ public class HourlyTeamScore extends UserScore {
c -> c.element().getValue()));
tableConfig.put("window_start",
new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
- c -> { IntervalWindow w = (IntervalWindow) c.window();
- return fmt.print(w.start()); }));
+ c -> {
+ IntervalWindow w = (IntervalWindow) c.window();
+ return fmt.print(w.start());
+ }));
return tableConfig;
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9e19efdf/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
----------------------------------------------------------------------
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 594d2b8..97958b0 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
@@ -143,8 +143,10 @@ public class LeaderBoard extends HourlyTeamScore {
c -> c.element().getValue()));
tableConfigure.put("window_start",
new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>("STRING",
- c -> { IntervalWindow w = (IntervalWindow) c.window();
- return fmt.print(w.start()); }));
+ c -> {
+ IntervalWindow w = (IntervalWindow) c.window();
+ return fmt.print(w.start());
+ }));
tableConfigure.put("processing_time",
new WriteWindowedToBigQuery.FieldInfo<KV<String, Integer>>(
"STRING", c -> fmt.print(Instant.now())));
[11/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java
deleted file mode 100644
index d2f8bbe..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/runners/DataflowPipelineRunner.java
+++ /dev/null
@@ -1,3022 +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.runners;
-
-import static org.apache.beam.sdk.util.StringUtils.approximatePTransformName;
-import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName;
-import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.Pipeline.PipelineVisitor;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.coders.AvroCoder;
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.coders.ListCoder;
-import org.apache.beam.sdk.coders.MapCoder;
-import org.apache.beam.sdk.coders.SerializableCoder;
-import org.apache.beam.sdk.coders.StandardCoder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.FileBasedSink;
-import org.apache.beam.sdk.io.PubsubIO;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.ShardNameTemplate;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.sdk.io.Write;
-import org.apache.beam.sdk.options.DataflowPipelineDebugOptions;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.DataflowPipelineWorkerPoolOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsValidator;
-import org.apache.beam.sdk.options.StreamingOptions;
-import org.apache.beam.sdk.runners.DataflowPipelineTranslator.JobSpecification;
-import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TransformTranslator;
-import org.apache.beam.sdk.runners.DataflowPipelineTranslator.TranslationContext;
-import org.apache.beam.sdk.runners.dataflow.AssignWindows;
-import org.apache.beam.sdk.runners.dataflow.DataflowAggregatorTransforms;
-import org.apache.beam.sdk.runners.dataflow.PubsubIOTranslator;
-import org.apache.beam.sdk.runners.dataflow.ReadTranslator;
-import org.apache.beam.sdk.runners.worker.IsmFormat;
-import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecord;
-import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecordCoder;
-import org.apache.beam.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
-import org.apache.beam.sdk.transforms.Aggregator;
-import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.Create;
-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.SerializableFunction;
-import org.apache.beam.sdk.transforms.View;
-import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
-import org.apache.beam.sdk.transforms.WithKeys;
-import org.apache.beam.sdk.transforms.windowing.AfterPane;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.DataflowTransport;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.util.InstanceBuilder;
-import org.apache.beam.sdk.util.MonitoringUtil;
-import org.apache.beam.sdk.util.PCollectionViews;
-import org.apache.beam.sdk.util.PathValidator;
-import org.apache.beam.sdk.util.PropertyNames;
-import org.apache.beam.sdk.util.ReleaseInfo;
-import org.apache.beam.sdk.util.Reshuffle;
-import org.apache.beam.sdk.util.SystemDoFnInternal;
-import org.apache.beam.sdk.util.ValueWithRecordId;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
-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.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PDone;
-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.TupleTag;
-import org.apache.beam.sdk.values.TupleTagList;
-
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.services.clouddebugger.v2.Clouddebugger;
-import com.google.api.services.clouddebugger.v2.model.Debuggee;
-import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest;
-import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.api.services.dataflow.model.Job;
-import com.google.api.services.dataflow.model.ListJobsResponse;
-import com.google.api.services.dataflow.model.WorkerPool;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.base.Joiner;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import com.google.common.base.Utf8;
-import com.google.common.collect.ForwardingMap;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multimap;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import org.joda.time.DateTimeUtils;
-import org.joda.time.DateTimeZone;
-import org.joda.time.Duration;
-import org.joda.time.format.DateTimeFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PrintWriter;
-import java.io.Serializable;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-/**
- * A {@link PipelineRunner} that executes the operations in the
- * pipeline by first translating them to the Dataflow representation
- * using the {@link DataflowPipelineTranslator} and then submitting
- * them to a Dataflow service for execution.
- *
- * <p><h3>Permissions</h3>
- * When reading from a Dataflow source or writing to a Dataflow sink using
- * {@code DataflowPipelineRunner}, the Google cloudservices account and the Google compute engine
- * service account of the GCP project running the Dataflow Job will need access to the corresponding
- * source/sink.
- *
- * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
- * Dataflow Security and Permissions</a> for more details.
- */
-public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob> {
- private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineRunner.class);
-
- /** Provided configuration options. */
- private final DataflowPipelineOptions options;
-
- /** Client for the Dataflow service. This is used to actually submit jobs. */
- private final Dataflow dataflowClient;
-
- /** Translator for this DataflowPipelineRunner, based on options. */
- private final DataflowPipelineTranslator translator;
-
- /** Custom transforms implementations. */
- private final Map<Class<?>, Class<?>> overrides;
-
- /** A set of user defined functions to invoke at different points in execution. */
- private DataflowPipelineRunnerHooks hooks;
-
- // Environment version information.
- private static final String ENVIRONMENT_MAJOR_VERSION = "4";
-
- // Default Docker container images that execute Dataflow worker harness, residing in Google
- // Container Registry, separately for Batch and Streaming.
- public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE
- = "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160422";
- public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE
- = "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160422";
-
- // The limit of CreateJob request size.
- private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024;
-
- private final Set<PCollection<?>> pcollectionsRequiringIndexedFormat;
-
- /**
- * Project IDs must contain lowercase letters, digits, or dashes.
- * IDs must start with a letter and may not end with a dash.
- * This regex isn't exact - this allows for patterns that would be rejected by
- * the service, but this is sufficient for basic validation of project IDs.
- */
- public static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]+[a-z0-9]";
-
- /**
- * Construct a runner from the provided options.
- *
- * @param options Properties that configure the runner.
- * @return The newly created runner.
- */
- public static DataflowPipelineRunner fromOptions(PipelineOptions options) {
- // (Re-)register standard IO factories. Clobbers any prior credentials.
- IOChannelUtils.registerStandardIOFactories(options);
-
- DataflowPipelineOptions dataflowOptions =
- PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options);
- ArrayList<String> missing = new ArrayList<>();
-
- if (dataflowOptions.getAppName() == null) {
- missing.add("appName");
- }
- if (missing.size() > 0) {
- throw new IllegalArgumentException(
- "Missing required values: " + Joiner.on(',').join(missing));
- }
-
- PathValidator validator = dataflowOptions.getPathValidator();
- Preconditions.checkArgument(!(Strings.isNullOrEmpty(dataflowOptions.getTempLocation())
- && Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())),
- "Missing required value: at least one of tempLocation or stagingLocation must be set.");
-
- if (dataflowOptions.getStagingLocation() != null) {
- validator.validateOutputFilePrefixSupported(dataflowOptions.getStagingLocation());
- }
- if (dataflowOptions.getTempLocation() != null) {
- validator.validateOutputFilePrefixSupported(dataflowOptions.getTempLocation());
- }
- if (Strings.isNullOrEmpty(dataflowOptions.getTempLocation())) {
- dataflowOptions.setTempLocation(dataflowOptions.getStagingLocation());
- } else if (Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())) {
- try {
- dataflowOptions.setStagingLocation(
- IOChannelUtils.resolve(dataflowOptions.getTempLocation(), "staging"));
- } catch (IOException e) {
- throw new IllegalArgumentException("Unable to resolve PipelineOptions.stagingLocation "
- + "from PipelineOptions.tempLocation. Please set the staging location explicitly.", e);
- }
- }
-
- if (dataflowOptions.getFilesToStage() == null) {
- dataflowOptions.setFilesToStage(detectClassPathResourcesToStage(
- DataflowPipelineRunner.class.getClassLoader()));
- LOG.info("PipelineOptions.filesToStage was not specified. "
- + "Defaulting to files from the classpath: will stage {} files. "
- + "Enable logging at DEBUG level to see which files will be staged.",
- dataflowOptions.getFilesToStage().size());
- LOG.debug("Classpath elements: {}", dataflowOptions.getFilesToStage());
- }
-
- // Verify jobName according to service requirements, truncating converting to lowercase if
- // necessary.
- String jobName =
- dataflowOptions
- .getJobName()
- .toLowerCase();
- checkArgument(
- jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"),
- "JobName invalid; the name must consist of only the characters "
- + "[-a-z0-9], starting with a letter and ending with a letter "
- + "or number");
- if (!jobName.equals(dataflowOptions.getJobName())) {
- LOG.info(
- "PipelineOptions.jobName did not match the service requirements. "
- + "Using {} instead of {}.",
- jobName,
- dataflowOptions.getJobName());
- }
- dataflowOptions.setJobName(jobName);
-
- // Verify project
- String project = dataflowOptions.getProject();
- if (project.matches("[0-9]*")) {
- throw new IllegalArgumentException("Project ID '" + project
- + "' invalid. Please make sure you specified the Project ID, not project number.");
- } else if (!project.matches(PROJECT_ID_REGEXP)) {
- throw new IllegalArgumentException("Project ID '" + project
- + "' invalid. Please make sure you specified the Project ID, not project description.");
- }
-
- DataflowPipelineDebugOptions debugOptions =
- dataflowOptions.as(DataflowPipelineDebugOptions.class);
- // Verify the number of worker threads is a valid value
- if (debugOptions.getNumberOfWorkerHarnessThreads() < 0) {
- throw new IllegalArgumentException("Number of worker harness threads '"
- + debugOptions.getNumberOfWorkerHarnessThreads()
- + "' invalid. Please make sure the value is non-negative.");
- }
-
- return new DataflowPipelineRunner(dataflowOptions);
- }
-
- @VisibleForTesting protected DataflowPipelineRunner(DataflowPipelineOptions options) {
- this.options = options;
- this.dataflowClient = options.getDataflowClient();
- this.translator = DataflowPipelineTranslator.fromOptions(options);
- this.pcollectionsRequiringIndexedFormat = new HashSet<>();
- this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>();
-
- if (options.isStreaming()) {
- overrides = ImmutableMap.<Class<?>, Class<?>>builder()
- .put(Combine.GloballyAsSingletonView.class, StreamingCombineGloballyAsSingletonView.class)
- .put(Create.Values.class, StreamingCreate.class)
- .put(View.AsMap.class, StreamingViewAsMap.class)
- .put(View.AsMultimap.class, StreamingViewAsMultimap.class)
- .put(View.AsSingleton.class, StreamingViewAsSingleton.class)
- .put(View.AsList.class, StreamingViewAsList.class)
- .put(View.AsIterable.class, StreamingViewAsIterable.class)
- .put(Write.Bound.class, StreamingWrite.class)
- .put(PubsubIO.Write.Bound.class, StreamingPubsubIOWrite.class)
- .put(Read.Unbounded.class, StreamingUnboundedRead.class)
- .put(Read.Bounded.class, UnsupportedIO.class)
- .put(AvroIO.Read.Bound.class, UnsupportedIO.class)
- .put(AvroIO.Write.Bound.class, UnsupportedIO.class)
- .put(BigQueryIO.Read.Bound.class, UnsupportedIO.class)
- .put(TextIO.Read.Bound.class, UnsupportedIO.class)
- .put(TextIO.Write.Bound.class, UnsupportedIO.class)
- .put(Window.Bound.class, AssignWindows.class)
- .build();
- } else {
- ImmutableMap.Builder<Class<?>, Class<?>> builder = ImmutableMap.<Class<?>, Class<?>>builder();
- builder.put(Read.Unbounded.class, UnsupportedIO.class);
- builder.put(Window.Bound.class, AssignWindows.class);
- builder.put(Write.Bound.class, BatchWrite.class);
- builder.put(AvroIO.Write.Bound.class, BatchAvroIOWrite.class);
- builder.put(TextIO.Write.Bound.class, BatchTextIOWrite.class);
- if (options.getExperiments() == null
- || !options.getExperiments().contains("disable_ism_side_input")) {
- builder.put(View.AsMap.class, BatchViewAsMap.class);
- builder.put(View.AsMultimap.class, BatchViewAsMultimap.class);
- builder.put(View.AsSingleton.class, BatchViewAsSingleton.class);
- builder.put(View.AsList.class, BatchViewAsList.class);
- builder.put(View.AsIterable.class, BatchViewAsIterable.class);
- }
- overrides = builder.build();
- }
- }
-
- /**
- * Applies the given transform to the input. For transforms with customized definitions
- * for the Dataflow pipeline runner, the application is intercepted and modified here.
- */
- @Override
- public <OutputT extends POutput, InputT extends PInput> OutputT apply(
- PTransform<InputT, OutputT> transform, InputT input) {
-
- if (Combine.GroupedValues.class.equals(transform.getClass())
- || GroupByKey.class.equals(transform.getClass())) {
-
- // For both Dataflow runners (streaming and batch), GroupByKey and GroupedValues are
- // primitives. Returning a primitive output instead of the expanded definition
- // signals to the translator that translation is necessary.
- @SuppressWarnings("unchecked")
- PCollection<?> pc = (PCollection<?>) input;
- @SuppressWarnings("unchecked")
- OutputT outputT = (OutputT) PCollection.createPrimitiveOutputInternal(
- pc.getPipeline(),
- transform instanceof GroupByKey
- ? ((GroupByKey<?, ?>) transform).updateWindowingStrategy(pc.getWindowingStrategy())
- : pc.getWindowingStrategy(),
- pc.isBounded());
- return outputT;
- } else if (Window.Bound.class.equals(transform.getClass())) {
- /*
- * TODO: make this the generic way overrides are applied (using super.apply() rather than
- * Pipeline.applyTransform(); this allows the apply method to be replaced without inserting
- * additional nodes into the graph.
- */
- // casting to wildcard
- @SuppressWarnings("unchecked")
- OutputT windowed = (OutputT) applyWindow((Window.Bound<?>) transform, (PCollection<?>) input);
- return windowed;
- } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
- && ((PCollectionList<?>) input).size() == 0) {
- return (OutputT) Pipeline.applyTransform(input, Create.of());
- } else if (overrides.containsKey(transform.getClass())) {
- // It is the responsibility of whoever constructs overrides to ensure this is type safe.
- @SuppressWarnings("unchecked")
- Class<PTransform<InputT, OutputT>> transformClass =
- (Class<PTransform<InputT, OutputT>>) transform.getClass();
-
- @SuppressWarnings("unchecked")
- Class<PTransform<InputT, OutputT>> customTransformClass =
- (Class<PTransform<InputT, OutputT>>) overrides.get(transform.getClass());
-
- PTransform<InputT, OutputT> customTransform =
- InstanceBuilder.ofType(customTransformClass)
- .withArg(DataflowPipelineRunner.class, this)
- .withArg(transformClass, transform)
- .build();
-
- return Pipeline.applyTransform(input, customTransform);
- } else {
- return super.apply(transform, input);
- }
- }
-
- private <T> PCollection<T> applyWindow(
- Window.Bound<?> intitialTransform, PCollection<?> initialInput) {
- // types are matched at compile time
- @SuppressWarnings("unchecked")
- Window.Bound<T> transform = (Window.Bound<T>) intitialTransform;
- @SuppressWarnings("unchecked")
- PCollection<T> input = (PCollection<T>) initialInput;
- return super.apply(new AssignWindows<>(transform), input);
- }
-
- private String debuggerMessage(String projectId, String uniquifier) {
- return String.format("To debug your job, visit Google Cloud Debugger at: "
- + "https://console.developers.google.com/debug?project=%s&dbgee=%s",
- projectId, uniquifier);
- }
-
- private void maybeRegisterDebuggee(DataflowPipelineOptions options, String uniquifier) {
- if (!options.getEnableCloudDebugger()) {
- return;
- }
-
- if (options.getDebuggee() != null) {
- throw new RuntimeException("Should not specify the debuggee");
- }
-
- Clouddebugger debuggerClient = DataflowTransport.newClouddebuggerClient(options).build();
- Debuggee debuggee = registerDebuggee(debuggerClient, uniquifier);
- options.setDebuggee(debuggee);
-
- System.out.println(debuggerMessage(options.getProject(), debuggee.getUniquifier()));
- }
-
- private Debuggee registerDebuggee(Clouddebugger debuggerClient, String uniquifier) {
- RegisterDebuggeeRequest registerReq = new RegisterDebuggeeRequest();
- registerReq.setDebuggee(new Debuggee()
- .setProject(options.getProject())
- .setUniquifier(uniquifier)
- .setDescription(uniquifier)
- .setAgentVersion("google.com/cloud-dataflow-java/v1"));
-
- try {
- RegisterDebuggeeResponse registerResponse =
- debuggerClient.controller().debuggees().register(registerReq).execute();
- Debuggee debuggee = registerResponse.getDebuggee();
- if (debuggee.getStatus() != null && debuggee.getStatus().getIsError()) {
- throw new RuntimeException("Unable to register with the debugger: " +
- debuggee.getStatus().getDescription().getFormat());
- }
-
- return debuggee;
- } catch (IOException e) {
- throw new RuntimeException("Unable to register with the debugger: ", e);
- }
- }
-
- @Override
- public DataflowPipelineJob run(Pipeline pipeline) {
- logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline);
-
- LOG.info("Executing pipeline on the Dataflow Service, which will have billing implications "
- + "related to Google Compute Engine usage and other Google Cloud Services.");
-
- List<DataflowPackage> packages = options.getStager().stageFiles();
-
-
- // Set a unique client_request_id in the CreateJob request.
- // This is used to ensure idempotence of job creation across retried
- // attempts to create a job. Specifically, if the service returns a job with
- // a different client_request_id, it means the returned one is a different
- // job previously created with the same job name, and that the job creation
- // has been effectively rejected. The SDK should return
- // Error::Already_Exists to user in that case.
- int randomNum = new Random().nextInt(9000) + 1000;
- String requestId = DateTimeFormat.forPattern("YYYYMMddHHmmssmmm").withZone(DateTimeZone.UTC)
- .print(DateTimeUtils.currentTimeMillis()) + "_" + randomNum;
-
- // Try to create a debuggee ID. This must happen before the job is translated since it may
- // update the options.
- DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
- maybeRegisterDebuggee(dataflowOptions, requestId);
-
- JobSpecification jobSpecification =
- translator.translate(pipeline, this, packages);
- Job newJob = jobSpecification.getJob();
- newJob.setClientRequestId(requestId);
-
- String version = ReleaseInfo.getReleaseInfo().getVersion();
- System.out.println("Dataflow SDK version: " + version);
-
- newJob.getEnvironment().setUserAgent(ReleaseInfo.getReleaseInfo());
- // The Dataflow Service may write to the temporary directory directly, so
- // must be verified.
- if (!Strings.isNullOrEmpty(options.getTempLocation())) {
- newJob.getEnvironment().setTempStoragePrefix(
- dataflowOptions.getPathValidator().verifyPath(options.getTempLocation()));
- }
- newJob.getEnvironment().setDataset(options.getTempDatasetId());
- newJob.getEnvironment().setExperiments(options.getExperiments());
-
- // Set the Docker container image that executes Dataflow worker harness, residing in Google
- // Container Registry. Translator is guaranteed to create a worker pool prior to this point.
- String workerHarnessContainerImage =
- options.as(DataflowPipelineWorkerPoolOptions.class)
- .getWorkerHarnessContainerImage();
- for (WorkerPool workerPool : newJob.getEnvironment().getWorkerPools()) {
- workerPool.setWorkerHarnessContainerImage(workerHarnessContainerImage);
- }
-
- // Requirements about the service.
- Map<String, Object> environmentVersion = new HashMap<>();
- environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, ENVIRONMENT_MAJOR_VERSION);
- newJob.getEnvironment().setVersion(environmentVersion);
- // Default jobType is JAVA_BATCH_AUTOSCALING: A Java job with workers that the job can
- // autoscale if specified.
- String jobType = "JAVA_BATCH_AUTOSCALING";
-
- if (options.isStreaming()) {
- jobType = "STREAMING";
- }
- environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType);
-
- if (hooks != null) {
- hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment());
- }
-
- if (!Strings.isNullOrEmpty(options.getDataflowJobFile())) {
- try (PrintWriter printWriter = new PrintWriter(
- new File(options.getDataflowJobFile()))) {
- String workSpecJson = DataflowPipelineTranslator.jobToString(newJob);
- printWriter.print(workSpecJson);
- LOG.info("Printed workflow specification to {}", options.getDataflowJobFile());
- } catch (IllegalStateException ex) {
- LOG.warn("Cannot translate workflow spec to json for debug.");
- } catch (FileNotFoundException ex) {
- LOG.warn("Cannot create workflow spec output file.");
- }
- }
-
- String jobIdToUpdate = null;
- if (options.isUpdate()) {
- jobIdToUpdate = getJobIdFromName(options.getJobName());
- newJob.setTransformNameMapping(options.getTransformNameMapping());
- newJob.setReplaceJobId(jobIdToUpdate);
- }
- Job jobResult;
- try {
- jobResult = dataflowClient
- .projects()
- .jobs()
- .create(options.getProject(), newJob)
- .execute();
- } catch (GoogleJsonResponseException e) {
- String errorMessages = "Unexpected errors";
- if (e.getDetails() != null) {
- if (Utf8.encodedLength(newJob.toString()) >= CREATE_JOB_REQUEST_LIMIT_BYTES) {
- errorMessages = "The size of the serialized JSON representation of the pipeline "
- + "exceeds the allowable limit. "
- + "For more information, please check the FAQ link below:\n"
- + "https://cloud.google.com/dataflow/faq";
- } else {
- errorMessages = e.getDetails().getMessage();
- }
- }
- throw new RuntimeException("Failed to create a workflow job: " + errorMessages, e);
- } catch (IOException e) {
- throw new RuntimeException("Failed to create a workflow job", e);
- }
-
- // Obtain all of the extractors from the PTransforms used in the pipeline so the
- // DataflowPipelineJob has access to them.
- AggregatorPipelineExtractor aggregatorExtractor = new AggregatorPipelineExtractor(pipeline);
- Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
- aggregatorExtractor.getAggregatorSteps();
-
- DataflowAggregatorTransforms aggregatorTransforms =
- new DataflowAggregatorTransforms(aggregatorSteps, jobSpecification.getStepNames());
-
- // Use a raw client for post-launch monitoring, as status calls may fail
- // regularly and need not be retried automatically.
- DataflowPipelineJob dataflowPipelineJob =
- new DataflowPipelineJob(options.getProject(), jobResult.getId(),
- DataflowTransport.newRawDataflowClient(options).build(), aggregatorTransforms);
-
- // If the service returned client request id, the SDK needs to compare it
- // with the original id generated in the request, if they are not the same
- // (i.e., the returned job is not created by this request), throw
- // DataflowJobAlreadyExistsException or DataflowJobAlreadyUpdatedExcetpion
- // depending on whether this is a reload or not.
- if (jobResult.getClientRequestId() != null && !jobResult.getClientRequestId().isEmpty()
- && !jobResult.getClientRequestId().equals(requestId)) {
- // If updating a job.
- if (options.isUpdate()) {
- throw new DataflowJobAlreadyUpdatedException(dataflowPipelineJob,
- String.format("The job named %s with id: %s has already been updated into job id: %s "
- + "and cannot be updated again.",
- newJob.getName(), jobIdToUpdate, jobResult.getId()));
- } else {
- throw new DataflowJobAlreadyExistsException(dataflowPipelineJob,
- String.format("There is already an active job named %s with id: %s. If you want "
- + "to submit a second job, try again by setting a different name using --jobName.",
- newJob.getName(), jobResult.getId()));
- }
- }
-
- LOG.info("To access the Dataflow monitoring console, please navigate to {}",
- MonitoringUtil.getJobMonitoringPageURL(options.getProject(), jobResult.getId()));
- System.out.println("Submitted job: " + jobResult.getId());
-
- LOG.info("To cancel the job using the 'gcloud' tool, run:\n> {}",
- MonitoringUtil.getGcloudCancelCommand(options, jobResult.getId()));
-
- return dataflowPipelineJob;
- }
-
- /**
- * Returns the DataflowPipelineTranslator associated with this object.
- */
- public DataflowPipelineTranslator getTranslator() {
- return translator;
- }
-
- /**
- * Sets callbacks to invoke during execution see {@code DataflowPipelineRunnerHooks}.
- */
- @Experimental
- public void setHooks(DataflowPipelineRunnerHooks hooks) {
- this.hooks = hooks;
- }
-
- /////////////////////////////////////////////////////////////////////////////
-
- /** Outputs a warning about PCollection views without deterministic key coders. */
- private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) {
- // We need to wait till this point to determine the names of the transforms since only
- // at this time do we know the hierarchy of the transforms otherwise we could
- // have just recorded the full names during apply time.
- if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) {
- final SortedSet<String> ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>();
- pipeline.traverseTopologically(new PipelineVisitor() {
- @Override
- public void visitValue(PValue value, TransformTreeNode producer) {
- }
-
- @Override
- public void visitTransform(TransformTreeNode node) {
- if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
- ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
- }
- }
-
- @Override
- public void enterCompositeTransform(TransformTreeNode node) {
- if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
- ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
- }
- }
-
- @Override
- public void leaveCompositeTransform(TransformTreeNode node) {
- }
- });
-
- LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} "
- + "because the key coder is not deterministic. Falling back to singleton implementation "
- + "which may cause memory and/or performance problems. Future major versions of "
- + "Dataflow will require deterministic key coders.",
- ptransformViewNamesWithNonDeterministicKeyCoders);
- }
- }
-
- /**
- * Returns true if the passed in {@link PCollection} needs to be materialiazed using
- * an indexed format.
- */
- boolean doesPCollectionRequireIndexedFormat(PCollection<?> pcol) {
- return pcollectionsRequiringIndexedFormat.contains(pcol);
- }
-
- /**
- * Marks the passed in {@link PCollection} as requiring to be materialized using
- * an indexed format.
- */
- private void addPCollectionRequiringIndexedFormat(PCollection<?> pcol) {
- pcollectionsRequiringIndexedFormat.add(pcol);
- }
-
- /** A set of {@link View}s with non-deterministic key coders. */
- Set<PTransform<?, ?>> ptransformViewsWithNonDeterministicKeyCoders;
-
- /**
- * Records that the {@link PTransform} requires a deterministic key coder.
- */
- private void recordViewUsesNonDeterministicKeyCoder(PTransform<?, ?> ptransform) {
- ptransformViewsWithNonDeterministicKeyCoders.add(ptransform);
- }
-
- /**
- * A {@link GroupByKey} transform for the {@link DataflowPipelineRunner} which sorts
- * values using the secondary key {@code K2}.
- *
- * <p>The {@link PCollection} created created by this {@link PTransform} will have values in
- * the empty window. Care must be taken *afterwards* to either re-window
- * (using {@link Window#into}) or only use {@link PTransform}s that do not depend on the
- * values being within a window.
- */
- static class GroupByKeyAndSortValuesOnly<K1, K2, V>
- extends PTransform<PCollection<KV<K1, KV<K2, V>>>, PCollection<KV<K1, Iterable<KV<K2, V>>>>> {
- private GroupByKeyAndSortValuesOnly() {
- }
-
- @Override
- public PCollection<KV<K1, Iterable<KV<K2, V>>>> apply(PCollection<KV<K1, KV<K2, V>>> input) {
- PCollection<KV<K1, Iterable<KV<K2, V>>>> rval =
- PCollection.<KV<K1, Iterable<KV<K2, V>>>>createPrimitiveOutputInternal(
- input.getPipeline(),
- WindowingStrategy.globalDefault(),
- IsBounded.BOUNDED);
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- KvCoder<K1, KV<K2, V>> inputCoder = (KvCoder) input.getCoder();
- rval.setCoder(
- KvCoder.of(inputCoder.getKeyCoder(),
- IterableCoder.of(inputCoder.getValueCoder())));
- return rval;
- }
- }
-
- /**
- * A {@link PTransform} that groups the values by a hash of the window's byte representation
- * and sorts the values using the windows byte representation.
- */
- private static class GroupByWindowHashAsKeyAndWindowAsSortKey<T, W extends BoundedWindow> extends
- PTransform<PCollection<T>, PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>>> {
-
- /**
- * A {@link DoFn} that for each element outputs a {@code KV} structure suitable for
- * grouping by the hash of the window's byte representation and sorting the grouped values
- * using the window's byte representation.
- */
- @SystemDoFnInternal
- private static class UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W extends BoundedWindow>
- extends DoFn<T, KV<Integer, KV<W, WindowedValue<T>>>> implements DoFn.RequiresWindowAccess {
-
- private final IsmRecordCoder<?> ismCoderForHash;
- private UseWindowHashAsKeyAndWindowAsSortKeyDoFn(IsmRecordCoder<?> ismCoderForHash) {
- this.ismCoderForHash = ismCoderForHash;
- }
-
- @Override
- public void processElement(ProcessContext c) throws Exception {
- @SuppressWarnings("unchecked")
- W window = (W) c.window();
- c.output(
- KV.of(ismCoderForHash.hash(ImmutableList.of(window)),
- KV.of(window,
- WindowedValue.of(
- c.element(),
- c.timestamp(),
- c.window(),
- c.pane()))));
- }
- }
-
- private final IsmRecordCoder<?> ismCoderForHash;
- private GroupByWindowHashAsKeyAndWindowAsSortKey(IsmRecordCoder<?> ismCoderForHash) {
- this.ismCoderForHash = ismCoderForHash;
- }
-
- @Override
- public PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>> apply(PCollection<T> input) {
- @SuppressWarnings("unchecked")
- Coder<W> windowCoder = (Coder<W>)
- input.getWindowingStrategy().getWindowFn().windowCoder();
- PCollection<KV<Integer, KV<W, WindowedValue<T>>>> rval =
- input.apply(ParDo.of(
- new UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W>(ismCoderForHash)));
- rval.setCoder(
- KvCoder.of(
- VarIntCoder.of(),
- KvCoder.of(windowCoder,
- FullWindowedValueCoder.of(input.getCoder(), windowCoder))));
- return rval.apply(new GroupByKeyAndSortValuesOnly<Integer, W, WindowedValue<T>>());
- }
- }
-
- /**
- * Specialized implementation for
- * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} for the
- * Dataflow runner in batch mode.
- *
- * <p>Creates a set of files in the {@link IsmFormat} sharded by the hash of the windows
- * byte representation and with records having:
- * <ul>
- * <li>Key 1: Window</li>
- * <li>Value: Windowed value</li>
- * </ul>
- */
- static class BatchViewAsSingleton<T>
- extends PTransform<PCollection<T>, PCollectionView<T>> {
-
- /**
- * A {@link DoFn} that outputs {@link IsmRecord}s. These records are structured as follows:
- * <ul>
- * <li>Key 1: Window
- * <li>Value: Windowed value
- * </ul>
- */
- static class IsmRecordForSingularValuePerWindowDoFn<T, W extends BoundedWindow>
- extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
- IsmRecord<WindowedValue<T>>> {
-
- @Override
- public void processElement(ProcessContext c) throws Exception {
- Iterator<KV<W, WindowedValue<T>>> iterator = c.element().getValue().iterator();
- while (iterator.hasNext()) {
- KV<W, WindowedValue<T>> next = iterator.next();
- c.output(
- IsmRecord.of(
- ImmutableList.of(next.getKey()), next.getValue()));
- }
- }
- }
-
- private final DataflowPipelineRunner runner;
- private final View.AsSingleton<T> transform;
- /**
- * Builds an instance of this class from the overridden transform.
- */
- @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
- public BatchViewAsSingleton(DataflowPipelineRunner runner, View.AsSingleton<T> transform) {
- this.runner = runner;
- this.transform = transform;
- }
-
- @Override
- public PCollectionView<T> apply(PCollection<T> input) {
- return BatchViewAsSingleton.<T, T, T, BoundedWindow>applyForSingleton(
- runner,
- input,
- new IsmRecordForSingularValuePerWindowDoFn<T, BoundedWindow>(),
- transform.hasDefaultValue(),
- transform.defaultValue(),
- input.getCoder());
- }
-
- static <T, FinalT, ViewT, W extends BoundedWindow> PCollectionView<ViewT>
- applyForSingleton(
- DataflowPipelineRunner runner,
- PCollection<T> input,
- DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
- IsmRecord<WindowedValue<FinalT>>> doFn,
- boolean hasDefault,
- FinalT defaultValue,
- Coder<FinalT> defaultValueCoder) {
-
- @SuppressWarnings("unchecked")
- Coder<W> windowCoder = (Coder<W>)
- input.getWindowingStrategy().getWindowFn().windowCoder();
-
- @SuppressWarnings({"rawtypes", "unchecked"})
- PCollectionView<ViewT> view =
- (PCollectionView<ViewT>) PCollectionViews.<FinalT, W>singletonView(
- input.getPipeline(),
- (WindowingStrategy) input.getWindowingStrategy(),
- hasDefault,
- defaultValue,
- defaultValueCoder);
-
- IsmRecordCoder<WindowedValue<FinalT>> ismCoder =
- coderForSingleton(windowCoder, defaultValueCoder);
-
- PCollection<IsmRecord<WindowedValue<FinalT>>> reifiedPerWindowAndSorted = input
- .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
- .apply(ParDo.of(doFn));
- reifiedPerWindowAndSorted.setCoder(ismCoder);
-
- runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
- return reifiedPerWindowAndSorted.apply(
- CreatePCollectionView.<IsmRecord<WindowedValue<FinalT>>, ViewT>of(view));
- }
-
- @Override
- protected String getKindString() {
- return "BatchViewAsSingleton";
- }
-
- static <T> IsmRecordCoder<WindowedValue<T>> coderForSingleton(
- Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
- return IsmRecordCoder.of(
- 1, // We hash using only the window
- 0, // There are no metadata records
- ImmutableList.<Coder<?>>of(windowCoder),
- FullWindowedValueCoder.of(valueCoder, windowCoder));
- }
- }
-
- /**
- * Specialized implementation for
- * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the
- * Dataflow runner in batch mode.
- *
- * <p>Creates a set of {@code Ism} files sharded by the hash of the windows byte representation
- * and with records having:
- * <ul>
- * <li>Key 1: Window</li>
- * <li>Key 2: Index offset within window</li>
- * <li>Value: Windowed value</li>
- * </ul>
- */
- static class BatchViewAsIterable<T>
- extends PTransform<PCollection<T>, PCollectionView<Iterable<T>>> {
-
- private final DataflowPipelineRunner runner;
- /**
- * Builds an instance of this class from the overridden transform.
- */
- @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
- public BatchViewAsIterable(DataflowPipelineRunner runner, View.AsIterable<T> transform) {
- this.runner = runner;
- }
-
- @Override
- public PCollectionView<Iterable<T>> apply(PCollection<T> input) {
- PCollectionView<Iterable<T>> view = PCollectionViews.iterableView(
- input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
- return BatchViewAsList.applyForIterableLike(runner, input, view);
- }
- }
-
- /**
- * Specialized implementation for
- * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the
- * Dataflow runner in batch mode.
- *
- * <p>Creates a set of {@code Ism} files sharded by the hash of the window's byte representation
- * and with records having:
- * <ul>
- * <li>Key 1: Window</li>
- * <li>Key 2: Index offset within window</li>
- * <li>Value: Windowed value</li>
- * </ul>
- */
- static class BatchViewAsList<T>
- extends PTransform<PCollection<T>, PCollectionView<List<T>>> {
- /**
- * A {@link DoFn} which creates {@link IsmRecord}s assuming that each element is within the
- * global window. Each {@link IsmRecord} has
- * <ul>
- * <li>Key 1: Global window</li>
- * <li>Key 2: Index offset within window</li>
- * <li>Value: Windowed value</li>
- * </ul>
- */
- @SystemDoFnInternal
- static class ToIsmRecordForGlobalWindowDoFn<T>
- extends DoFn<T, IsmRecord<WindowedValue<T>>> {
-
- long indexInBundle;
- @Override
- public void startBundle(Context c) throws Exception {
- indexInBundle = 0;
- }
-
- @Override
- public void processElement(ProcessContext c) throws Exception {
- c.output(IsmRecord.of(
- ImmutableList.of(GlobalWindow.INSTANCE, indexInBundle),
- WindowedValue.of(
- c.element(),
- c.timestamp(),
- GlobalWindow.INSTANCE,
- c.pane())));
- indexInBundle += 1;
- }
- }
-
- /**
- * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
- * to locate the window boundaries. The {@link IsmRecord} has:
- * <ul>
- * <li>Key 1: Window</li>
- * <li>Key 2: Index offset within window</li>
- * <li>Value: Windowed value</li>
- * </ul>
- */
- @SystemDoFnInternal
- static class ToIsmRecordForNonGlobalWindowDoFn<T, W extends BoundedWindow>
- extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
- IsmRecord<WindowedValue<T>>> {
-
- private final Coder<W> windowCoder;
- ToIsmRecordForNonGlobalWindowDoFn(Coder<W> windowCoder) {
- this.windowCoder = windowCoder;
- }
-
- @Override
- public void processElement(ProcessContext c) throws Exception {
- long elementsInWindow = 0;
- Optional<Object> previousWindowStructuralValue = Optional.absent();
- for (KV<W, WindowedValue<T>> value : c.element().getValue()) {
- Object currentWindowStructuralValue = windowCoder.structuralValue(value.getKey());
- // Compare to see if this is a new window so we can reset the index counter i
- if (previousWindowStructuralValue.isPresent()
- && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
- // Reset i since we have a new window.
- elementsInWindow = 0;
- }
- c.output(IsmRecord.of(
- ImmutableList.of(value.getKey(), elementsInWindow),
- value.getValue()));
- previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
- elementsInWindow += 1;
- }
- }
- }
-
- private final DataflowPipelineRunner runner;
- /**
- * Builds an instance of this class from the overridden transform.
- */
- @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
- public BatchViewAsList(DataflowPipelineRunner runner, View.AsList<T> transform) {
- this.runner = runner;
- }
-
- @Override
- public PCollectionView<List<T>> apply(PCollection<T> input) {
- PCollectionView<List<T>> view = PCollectionViews.listView(
- input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
- return applyForIterableLike(runner, input, view);
- }
-
- static <T, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForIterableLike(
- DataflowPipelineRunner runner,
- PCollection<T> input,
- PCollectionView<ViewT> view) {
-
- @SuppressWarnings("unchecked")
- Coder<W> windowCoder = (Coder<W>)
- input.getWindowingStrategy().getWindowFn().windowCoder();
-
- IsmRecordCoder<WindowedValue<T>> ismCoder = coderForListLike(windowCoder, input.getCoder());
-
- // If we are working in the global window, we do not need to do a GBK using the window
- // as the key since all the elements of the input PCollection are already such.
- // We just reify the windowed value while converting them to IsmRecords and generating
- // an index based upon where we are within the bundle. Each bundle
- // maps to one file exactly.
- if (input.getWindowingStrategy().getWindowFn() instanceof GlobalWindows) {
- PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted =
- input.apply(ParDo.of(new ToIsmRecordForGlobalWindowDoFn<T>()));
- reifiedPerWindowAndSorted.setCoder(ismCoder);
-
- runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
- return reifiedPerWindowAndSorted.apply(
- CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
- }
-
- PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted = input
- .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
- .apply(ParDo.of(new ToIsmRecordForNonGlobalWindowDoFn<T, W>(windowCoder)));
- reifiedPerWindowAndSorted.setCoder(ismCoder);
-
- runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
- return reifiedPerWindowAndSorted.apply(
- CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
- }
-
- @Override
- protected String getKindString() {
- return "BatchViewAsList";
- }
-
- static <T> IsmRecordCoder<WindowedValue<T>> coderForListLike(
- Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
- // TODO: swap to use a variable length long coder which has values which compare
- // the same as their byte representation compare lexicographically within the key coder
- return IsmRecordCoder.of(
- 1, // We hash using only the window
- 0, // There are no metadata records
- ImmutableList.of(windowCoder, BigEndianLongCoder.of()),
- FullWindowedValueCoder.of(valueCoder, windowCoder));
- }
- }
-
- /**
- * Specialized implementation for
- * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} for the
- * Dataflow runner in batch mode.
- *
- * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
- * representation. Each record is structured as follows:
- * <ul>
- * <li>Key 1: User key K</li>
- * <li>Key 2: Window</li>
- * <li>Key 3: 0L (constant)</li>
- * <li>Value: Windowed value</li>
- * </ul>
- *
- * <p>Alongside the data records, there are the following metadata records:
- * <ul>
- * <li>Key 1: Metadata Key</li>
- * <li>Key 2: Window</li>
- * <li>Key 3: Index [0, size of map]</li>
- * <li>Value: variable length long byte representation of size of map if index is 0,
- * otherwise the byte representation of a key</li>
- * </ul>
- * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
- * {@code [META, Window, i]} for {@code i} in {@code [1, size of map]} stores a the users key.
- * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
- * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
- * {@code [1, size of map]}.
- *
- * <p>Note that in the case of a non-deterministic key coder, we fallback to using
- * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing
- * a warning to users to specify a deterministic key coder.
- */
- static class BatchViewAsMap<K, V>
- extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, V>>> {
-
- /**
- * A {@link DoFn} which groups elements by window boundaries. For each group,
- * the group of elements is transformed into a {@link TransformedMap}.
- * The transformed {@code Map<K, V>} is backed by a {@code Map<K, WindowedValue<V>>}
- * and contains a function {@code WindowedValue<V> -> V}.
- *
- * <p>Outputs {@link IsmRecord}s having:
- * <ul>
- * <li>Key 1: Window</li>
- * <li>Value: Transformed map containing a transform that removes the encapsulation
- * of the window around each value,
- * {@code Map<K, WindowedValue<V>> -> Map<K, V>}.</li>
- * </ul>
- */
- static class ToMapDoFn<K, V, W extends BoundedWindow>
- extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
- IsmRecord<WindowedValue<TransformedMap<K,
- WindowedValue<V>,
- V>>>> {
-
- private final Coder<W> windowCoder;
- ToMapDoFn(Coder<W> windowCoder) {
- this.windowCoder = windowCoder;
- }
-
- @Override
- public void processElement(ProcessContext c)
- throws Exception {
- Optional<Object> previousWindowStructuralValue = Optional.absent();
- Optional<W> previousWindow = Optional.absent();
- Map<K, WindowedValue<V>> map = new HashMap<>();
- for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
- Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
- if (previousWindowStructuralValue.isPresent()
- && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
- // Construct the transformed map containing all the elements since we
- // are at a window boundary.
- c.output(IsmRecord.of(
- ImmutableList.of(previousWindow.get()),
- valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
- map = new HashMap<>();
- }
-
- // Verify that the user isn't trying to insert the same key multiple times.
- checkState(!map.containsKey(kv.getValue().getValue().getKey()),
- "Multiple values [%s, %s] found for single key [%s] within window [%s].",
- map.get(kv.getValue().getValue().getKey()),
- kv.getValue().getValue().getValue(),
- kv.getKey());
- map.put(kv.getValue().getValue().getKey(),
- kv.getValue().withValue(kv.getValue().getValue().getValue()));
- previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
- previousWindow = Optional.of(kv.getKey());
- }
-
- // The last value for this hash is guaranteed to be at a window boundary
- // so we output a transformed map containing all the elements since the last
- // window boundary.
- c.output(IsmRecord.of(
- ImmutableList.of(previousWindow.get()),
- valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
- }
- }
-
- private final DataflowPipelineRunner runner;
- /**
- * Builds an instance of this class from the overridden transform.
- */
- @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
- public BatchViewAsMap(DataflowPipelineRunner runner, View.AsMap<K, V> transform) {
- this.runner = runner;
- }
-
- @Override
- public PCollectionView<Map<K, V>> apply(PCollection<KV<K, V>> input) {
- return this.<BoundedWindow>applyInternal(input);
- }
-
- private <W extends BoundedWindow> PCollectionView<Map<K, V>>
- applyInternal(PCollection<KV<K, V>> input) {
-
- @SuppressWarnings({"rawtypes", "unchecked"})
- KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
- try {
- PCollectionView<Map<K, V>> view = PCollectionViews.mapView(
- input.getPipeline(), input.getWindowingStrategy(), inputCoder);
- return BatchViewAsMultimap.applyForMapLike(runner, input, view, true /* unique keys */);
- } catch (NonDeterministicException e) {
- runner.recordViewUsesNonDeterministicKeyCoder(this);
-
- // Since the key coder is not deterministic, we convert the map into a singleton
- // and return a singleton view equivalent.
- return applyForSingletonFallback(input);
- }
- }
-
- @Override
- protected String getKindString() {
- return "BatchViewAsMap";
- }
-
- /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
- private <W extends BoundedWindow> PCollectionView<Map<K, V>>
- applyForSingletonFallback(PCollection<KV<K, V>> input) {
- @SuppressWarnings("unchecked")
- Coder<W> windowCoder = (Coder<W>)
- input.getWindowingStrategy().getWindowFn().windowCoder();
-
- @SuppressWarnings({"rawtypes", "unchecked"})
- KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- Coder<Function<WindowedValue<V>, V>> transformCoder =
- (Coder) SerializableCoder.of(WindowedValueToValue.class);
-
- Coder<TransformedMap<K, WindowedValue<V>, V>> finalValueCoder =
- TransformedMapCoder.of(
- transformCoder,
- MapCoder.of(
- inputCoder.getKeyCoder(),
- FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder)));
-
- TransformedMap<K, WindowedValue<V>, V> defaultValue = new TransformedMap<>(
- WindowedValueToValue.<V>of(),
- ImmutableMap.<K, WindowedValue<V>>of());
-
- return BatchViewAsSingleton.<KV<K, V>,
- TransformedMap<K, WindowedValue<V>, V>,
- Map<K, V>,
- W> applyForSingleton(
- runner,
- input,
- new ToMapDoFn<K, V, W>(windowCoder),
- true,
- defaultValue,
- finalValueCoder);
- }
- }
-
- /**
- * Specialized implementation for
- * {@link org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the
- * Dataflow runner in batch mode.
- *
- * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
- * representation. Each record is structured as follows:
- * <ul>
- * <li>Key 1: User key K</li>
- * <li>Key 2: Window</li>
- * <li>Key 3: Index offset for a given key and window.</li>
- * <li>Value: Windowed value</li>
- * </ul>
- *
- * <p>Alongside the data records, there are the following metadata records:
- * <ul>
- * <li>Key 1: Metadata Key</li>
- * <li>Key 2: Window</li>
- * <li>Key 3: Index [0, size of map]</li>
- * <li>Value: variable length long byte representation of size of map if index is 0,
- * otherwise the byte representation of a key</li>
- * </ul>
- * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
- * {@code [META, Window, i]} for {@code i} in {@code [1, size of map]} stores a the users key.
- * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
- * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
- * {@code [1, size of map]}.
- *
- * <p>Note that in the case of a non-deterministic key coder, we fallback to using
- * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing
- * a warning to users to specify a deterministic key coder.
- */
- static class BatchViewAsMultimap<K, V>
- extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, Iterable<V>>>> {
- /**
- * A {@link PTransform} that groups elements by the hash of window's byte representation
- * if the input {@link PCollection} is not within the global window. Otherwise by the hash
- * of the window and key's byte representation. This {@link PTransform} also sorts
- * the values by the combination of the window and key's byte representations.
- */
- private static class GroupByKeyHashAndSortByKeyAndWindow<K, V, W extends BoundedWindow>
- extends PTransform<PCollection<KV<K, V>>,
- PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>> {
-
- @SystemDoFnInternal
- private static class GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>
- extends DoFn<KV<K, V>, KV<Integer, KV<KV<K, W>, WindowedValue<V>>>>
- implements DoFn.RequiresWindowAccess {
-
- private final IsmRecordCoder<?> coder;
- private GroupByKeyHashAndSortByKeyAndWindowDoFn(IsmRecordCoder<?> coder) {
- this.coder = coder;
- }
-
- @Override
- public void processElement(ProcessContext c) throws Exception {
- @SuppressWarnings("unchecked")
- W window = (W) c.window();
-
- c.output(
- KV.of(coder.hash(ImmutableList.of(c.element().getKey())),
- KV.of(KV.of(c.element().getKey(), window),
- WindowedValue.of(
- c.element().getValue(),
- c.timestamp(),
- (BoundedWindow) window,
- c.pane()))));
- }
- }
-
- private final IsmRecordCoder<?> coder;
- public GroupByKeyHashAndSortByKeyAndWindow(IsmRecordCoder<?> coder) {
- this.coder = coder;
- }
-
- @Override
- public PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>
- apply(PCollection<KV<K, V>> input) {
-
- @SuppressWarnings("unchecked")
- Coder<W> windowCoder = (Coder<W>)
- input.getWindowingStrategy().getWindowFn().windowCoder();
- @SuppressWarnings("unchecked")
- KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
-
- PCollection<KV<Integer, KV<KV<K, W>, WindowedValue<V>>>> keyedByHash;
- keyedByHash = input.apply(
- ParDo.of(new GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>(coder)));
- keyedByHash.setCoder(
- KvCoder.of(
- VarIntCoder.of(),
- KvCoder.of(KvCoder.of(inputCoder.getKeyCoder(), windowCoder),
- FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
-
- return keyedByHash.apply(
- new GroupByKeyAndSortValuesOnly<Integer, KV<K, W>, WindowedValue<V>>());
- }
- }
-
- /**
- * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
- * and keys to locate window and key boundaries. The main output {@link IsmRecord}s have:
- * <ul>
- * <li>Key 1: Window</li>
- * <li>Key 2: User key K</li>
- * <li>Key 3: Index offset for a given key and window.</li>
- * <li>Value: Windowed value</li>
- * </ul>
- *
- * <p>Additionally, we output all the unique keys per window seen to {@code outputForEntrySet}
- * and the unique key count per window to {@code outputForSize}.
- *
- * <p>Finally, if this DoFn has been requested to perform unique key checking, it will
- * throw an {@link IllegalStateException} if more than one key per window is found.
- */
- static class ToIsmRecordForMapLikeDoFn<K, V, W extends BoundedWindow>
- extends DoFn<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>,
- IsmRecord<WindowedValue<V>>> {
-
- private final TupleTag<KV<Integer, KV<W, Long>>> outputForSize;
- private final TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet;
- private final Coder<W> windowCoder;
- private final Coder<K> keyCoder;
- private final IsmRecordCoder<WindowedValue<V>> ismCoder;
- private final boolean uniqueKeysExpected;
- ToIsmRecordForMapLikeDoFn(
- TupleTag<KV<Integer, KV<W, Long>>> outputForSize,
- TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet,
- Coder<W> windowCoder,
- Coder<K> keyCoder,
- IsmRecordCoder<WindowedValue<V>> ismCoder,
- boolean uniqueKeysExpected) {
- this.outputForSize = outputForSize;
- this.outputForEntrySet = outputForEntrySet;
- this.windowCoder = windowCoder;
- this.keyCoder = keyCoder;
- this.ismCoder = ismCoder;
- this.uniqueKeysExpected = uniqueKeysExpected;
- }
-
- @Override
- public void processElement(ProcessContext c) throws Exception {
- long currentKeyIndex = 0;
- // We use one based indexing while counting
- long currentUniqueKeyCounter = 1;
- Iterator<KV<KV<K, W>, WindowedValue<V>>> iterator = c.element().getValue().iterator();
-
- KV<KV<K, W>, WindowedValue<V>> currentValue = iterator.next();
- Object currentKeyStructuralValue =
- keyCoder.structuralValue(currentValue.getKey().getKey());
- Object currentWindowStructuralValue =
- windowCoder.structuralValue(currentValue.getKey().getValue());
-
- while (iterator.hasNext()) {
- KV<KV<K, W>, WindowedValue<V>> nextValue = iterator.next();
- Object nextKeyStructuralValue =
- keyCoder.structuralValue(nextValue.getKey().getKey());
- Object nextWindowStructuralValue =
- windowCoder.structuralValue(nextValue.getKey().getValue());
-
- outputDataRecord(c, currentValue, currentKeyIndex);
-
- final long nextKeyIndex;
- final long nextUniqueKeyCounter;
-
- // Check to see if its a new window
- if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
- // The next value is a new window, so we output for size the number of unique keys
- // seen and the last key of the window. We also reset the next key index the unique
- // key counter.
- outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
- outputMetadataRecordForEntrySet(c, currentValue);
-
- nextKeyIndex = 0;
- nextUniqueKeyCounter = 1;
- } else if (!currentKeyStructuralValue.equals(nextKeyStructuralValue)){
- // It is a new key within the same window so output the key for the entry set,
- // reset the key index and increase the count of unique keys seen within this window.
- outputMetadataRecordForEntrySet(c, currentValue);
-
- nextKeyIndex = 0;
- nextUniqueKeyCounter = currentUniqueKeyCounter + 1;
- } else if (!uniqueKeysExpected) {
- // It is not a new key so we don't have to output the number of elements in this
- // window or increase the unique key counter. All we do is increase the key index.
-
- nextKeyIndex = currentKeyIndex + 1;
- nextUniqueKeyCounter = currentUniqueKeyCounter;
- } else {
- throw new IllegalStateException(String.format(
- "Unique keys are expected but found key %s with values %s and %s in window %s.",
- currentValue.getKey().getKey(),
- currentValue.getValue().getValue(),
- nextValue.getValue().getValue(),
- currentValue.getKey().getValue()));
- }
-
- currentValue = nextValue;
- currentWindowStructuralValue = nextWindowStructuralValue;
- currentKeyStructuralValue = nextKeyStructuralValue;
- currentKeyIndex = nextKeyIndex;
- currentUniqueKeyCounter = nextUniqueKeyCounter;
- }
-
- outputDataRecord(c, currentValue, currentKeyIndex);
- outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
- // The last value for this hash is guaranteed to be at a window boundary
- // so we output a record with the number of unique keys seen.
- outputMetadataRecordForEntrySet(c, currentValue);
- }
-
- /** This outputs the data record. */
- private void outputDataRecord(
- ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long keyIndex) {
- IsmRecord<WindowedValue<V>> ismRecord = IsmRecord.of(
- ImmutableList.of(
- value.getKey().getKey(),
- value.getKey().getValue(),
- keyIndex),
- value.getValue());
- c.output(ismRecord);
- }
-
- /**
- * This outputs records which will be used to compute the number of keys for a given window.
- */
- private void outputMetadataRecordForSize(
- ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long uniqueKeyCount) {
- c.sideOutput(outputForSize,
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
- value.getKey().getValue())),
- KV.of(value.getKey().getValue(), uniqueKeyCount)));
- }
-
- /** This outputs records which will be used to construct the entry set. */
- private void outputMetadataRecordForEntrySet(
- ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value) {
- c.sideOutput(outputForEntrySet,
- KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
- value.getKey().getValue())),
- KV.of(value.getKey().getValue(), value.getKey().getKey())));
- }
- }
-
- /**
- * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window of:
- * <ul>
- * <li>Key 1: META key</li>
- * <li>Key 2: window</li>
- * <li>Key 3: 0L (constant)</li>
- * <li>Value: sum of values for window</li>
- * </ul>
- *
- * <p>This {@link DoFn} is meant to be used to compute the number of unique keys
- * per window for map and multimap side inputs.
- */
- static class ToIsmMetadataRecordForSizeDoFn<K, V, W extends BoundedWindow>
- extends DoFn<KV<Integer, Iterable<KV<W, Long>>>, IsmRecord<WindowedValue<V>>> {
- private final Coder<W> windowCoder;
- ToIsmMetadataRecordForSizeDoFn(Coder<W> windowCoder) {
- this.windowCoder = windowCoder;
- }
-
- @Override
- public void processElement(ProcessContext c) throws Exception {
- Iterator<KV<W, Long>> iterator = c.element().getValue().iterator();
- KV<W, Long> currentValue = iterator.next();
- Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
- long size = 0;
- while (iterator.hasNext()) {
- KV<W, Long> nextValue = iterator.next();
- Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
-
- size += currentValue.getValue();
- if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
- c.output(IsmRecord.<WindowedValue<V>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
- CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
- size = 0;
- }
-
- currentValue = nextValue;
- currentWindowStructuralValue = nextWindowStructuralValue;
- }
-
- size += currentValue.getValue();
- // Output the final value since it is guaranteed to be on a window boundary.
- c.output(IsmRecord.<WindowedValue<V>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
- CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
- }
- }
-
- /**
- * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window and key pair of:
- * <ul>
- * <li>Key 1: META key</li>
- * <li>Key 2: window</li>
- * <li>Key 3: index offset (1-based index)</li>
- * <li>Value: key</li>
- * </ul>
- *
- * <p>This {@link DoFn} is meant to be used to output index to key records
- * per window for map and multimap side inputs.
- */
- static class ToIsmMetadataRecordForKeyDoFn<K, V, W extends BoundedWindow>
- extends DoFn<KV<Integer, Iterable<KV<W, K>>>, IsmRecord<WindowedValue<V>>> {
-
- private final Coder<K> keyCoder;
- private final Coder<W> windowCoder;
- ToIsmMetadataRecordForKeyDoFn(Coder<K> keyCoder, Coder<W> windowCoder) {
- this.keyCoder = keyCoder;
- this.windowCoder = windowCoder;
- }
-
- @Override
- public void processElement(ProcessContext c) throws Exception {
- Iterator<KV<W, K>> iterator = c.element().getValue().iterator();
- KV<W, K> currentValue = iterator.next();
- Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
- long elementsInWindow = 1;
- while (iterator.hasNext()) {
- KV<W, K> nextValue = iterator.next();
- Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
-
- c.output(IsmRecord.<WindowedValue<V>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
- CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
- elementsInWindow += 1;
-
- if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
- elementsInWindow = 1;
- }
-
- currentValue = nextValue;
- currentWindowStructuralValue = nextWindowStructuralValue;
- }
-
- // Output the final value since it is guaranteed to be on a window boundary.
- c.output(IsmRecord.<WindowedValue<V>>meta(
- ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
- CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
- }
- }
-
- /**
- * A {@link DoFn} which partitions sets of elements by window boundaries. Within each
- * partition, the set of elements is transformed into a {@link TransformedMap}.
- * The transformed {@code Map<K, Iterable<V>>} is backed by a
- * {@code Map<K, Iterable<WindowedValue<V>>>} and contains a function
- * {@code Iterable<WindowedValue<V>> -> Iterable<V>}.
- *
- * <p>Outputs {@link IsmRecord}s having:
- * <ul>
- * <li>Key 1: Window</li>
- * <li>Value: Transformed map containing a transform that removes the encapsulation
- * of the window around each value,
- * {@code Map<K, Iterable<WindowedValue<V>>> -> Map<K, Iterable<V>>}.</li>
- * </ul>
- */
- static class ToMultimapDoFn<K, V, W extends BoundedWindow>
- extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
- IsmRecord<WindowedValue<TransformedMap<K,
- Iterable<WindowedValue<V>>,
- Iterable<V>>>>> {
-
- private final Coder<W> windowCoder;
- ToMultimapDoFn(Coder<W> windowCoder) {
- this.windowCoder = windowCoder;
- }
-
- @Override
- public void processElement(ProcessContext c)
- throws Exception {
- Optional<Object> previousWindowStructuralValue = Optional.absent();
- Optional<W> previousWindow = Optional.absent();
- Multimap<K, WindowedValue<V>> multimap = HashMultimap.create();
- for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
- Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
- if (previousWindowStructuralValue.isPresent()
- && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
- // Construct the transformed map containing all the elements since we
- // are at a window boundary.
- @SuppressWarnings({"unchecked", "rawtypes"})
- Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
- c.output(IsmRecord.<WindowedValue<TransformedMap<K,
- Iterable<WindowedValue<V>>,
- Iterable<V>>>>of(
- ImmutableList.of(previousWindow.get()),
- valueInEmptyWindows(
- new TransformedMap<>(
- IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
- multimap = HashMultimap.create();
- }
-
- multimap.put(kv.getValue().getValue().getKey(),
- kv.getValue().withValue(kv.getValue().getValue().getValue()));
- previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
- previousWindow = Optional.of(kv.getKey());
- }
-
- // The last value for this hash is guaranteed to be at a window boundary
- // so we output a transformed map containing all the elements since the last
- // window boundary.
- @SuppressWarnings({"unchecked", "rawtypes"})
- Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
- c.output(IsmRecord.<WindowedValue<TransformedMap<K,
- Iterable<WindowedValue<V>>,
- Iterable<V>>>>of(
- ImmutableList.of(previousWindow.get()),
- valueInEmptyWindows(
- new TransformedMap<>(IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
- }
- }
-
- private final DataflowPipelineRunner runner;
- /**
- * Builds an instance of this class from the overridden transform.
- */
- @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
- public BatchViewAsMultimap(DataflowPipelineRunner runner, View.AsMultimap<K, V> transform) {
- this.runner = runner;
- }
-
- @Override
- public PCollectionView<Map<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
- return this.<BoundedWindow>applyInternal(input);
- }
-
- private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
- applyInternal(PCollection<KV<K, V>> input) {
- @SuppressWarnings({"rawtypes", "unchecked"})
- KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
- try {
- PCollectionView<Map<K, Iterable<V>>> view = PCollectionViews.multimapView(
- input.getPipeline(), input.getWindowingStrategy(), inputCoder);
-
- return applyForMapLike(runner, input, view, false /* unique keys not expected */);
- } catch (NonDeterministicException e) {
- runner.recordViewUsesNonDeterministicKeyCoder(this);
-
- // Since the key coder is not deterministic, we convert the map into a singleton
- // and return a singleton view equivalent.
- return applyForSingletonFallback(input);
- }
- }
-
- /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
- private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
- applyForSingletonFallback(PCollection<KV<K, V>> input) {
- @SuppressWarnings("unchecked")
- Coder<W> windowCoder = (Coder<W>)
- input.getWindowingStrategy().getWindowFn().windowCoder();
-
- @SuppressWarnings({"rawtypes", "unchecked"})
- KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-
- @SuppressWarnings({"unchecked", "rawtypes"})
- Coder<Function<Iterable<WindowedValue<V>>, Iterable<V>>> transformCoder =
- (Coder) SerializableCoder.of(IterableWithWindowedValuesToIterable.class);
-
- Coder<TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>> finalValueCoder =
- TransformedMapCoder.of(
- transformCoder,
- MapCoder.of(
- inputCoder.getKeyCoder(),
- IterableCoder.of(
- FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
-
- TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>> defaultValue =
- new TransformedMap<>(
- IterableWithWindowedValuesToIterable.<V>of(),
- ImmutableMap.<K, Iterable<WindowedValue<V>>>of());
-
- return BatchViewAsSingleton.<KV<K, V>,
- TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>,
- Map<K, Iterable<V>>,
- W> applyForSingleton(
- runner,
- input,
- new ToMultimapDoFn<K, V, W>(windowCoder),
- true,
- defaultValue,
- finalValueCoder);
- }
-
- private static <K, V, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForMapLike(
- DataflowPipelineRunner runner,
- PCollection<KV<K, V>> input,
- PCollectionView<ViewT> view,
- boolean uniqueKeysExpected) throws NonDeterministicException {
-
- @SuppressWarnings("unchecked")
- Coder<W> windowCoder = (Coder<W>)
- input.getWindowingStrategy().getWindowFn().windowCoder();
-
- @SuppressWarnings({"rawtypes", "unchecked"})
- KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
-
- // If our key coder is deterministic, we can use the key portion of each KV
- // part of a composite key containing the window , key and index.
- inputCoder.getKeyCoder().verifyDeterministic();
-
- IsmRecordCoder<WindowedValue<V>> ismCoder =
- coderForMapLike(windowCoder, inputCoder.getKeyCoder(), inputCoder.getValueCoder());
-
- // Create the various output tags representing the main output containing the data stream
- // and the side outputs containing the metadata about the size and entry set.
- TupleTag<IsmRecord<WindowedValue<V>>> mainOutputTag = new TupleTag<>();
- TupleTag<KV<Integer, KV<W, Long>>> outputForSizeTag = new TupleTag<>();
- TupleTag<KV<Integer, KV<W, K>>> outputForEntrySetTag = new TupleTag<>();
-
- // Process all the elements grouped by key hash, and sorted by key and then window
- // outputting to all the outputs defined above.
- PCollectionTuple outputTuple = input
- .apply("GBKaSVForData", new GroupByKeyHashAndSortByKeyAndWindow<K, V, W>(ismCoder))
- .apply(ParDo.of(new ToIsmRecordForMapLikeDoFn<K, V, W>(
- outputForSizeTag, outputForEntrySetTag,
- windowCoder, inputCoder.getKeyCoder(), ismCoder, uniqueKeysExpected))
- .withOutputTags(mainOutputTag,
- TupleTagList.of(
- ImmutableList.<TupleTag<?>>of(outputForSizeTag,
- outputForEntrySetTag))));
-
- // Set the coder on the main data output.
- PCollection<IsmRecord<WindowedValue<V>>> perHashWithReifiedWindows =
- outputTuple.get(mainOutputTag);
- perHashWithReifiedWindows.setCoder(ismCoder);
-
- // Set the coder on the metadata output for size and process the entries
- // producing a [META, Window, 0L] record per window storing the number of unique keys
- // for each window.
- PCollection<KV<Integer, KV<W, Long>>> outputForSize = outputTuple.get(outputForSizeTag);
- outputForSize.setCoder(
- KvCoder.of(VarIntCoder.of(),
- KvCoder.of(windowCoder, VarLongCoder.of())));
- PCollection<IsmRecord<WindowedValue<V>>> windowMapSizeMetadata = outputForSize
- .apply("GBKaSVForSize", new GroupByKeyAndSortValuesOnly<Integer, W, Long>())
- .apply(ParDo.of(new ToIsmMetadataRecordForSizeDoFn<K, V, W>(windowCoder)));
- windowMapSizeMetadata.setCoder(ismCoder);
-
- // Set the coder on the metadata output destined to build the entry set and process the
- // entries producing a [META, Window, Index] record per window key pair storing the key.
- PCollection<KV<Integer, KV<W, K>>> outputForEntrySet =
- outputTuple.get(outputForEntrySetTag);
- outputForEntrySet.setCoder(
- KvCoder.of(VarIntCoder.of(),
- KvCoder.of(windowCoder, inputCoder.getKeyCoder())));
- PCollection<IsmRecord<WindowedValue<V>>> windowMapKeysMetadata = outputForEntrySet
- .apply("GBKaSVForKeys", new GroupByKeyAndSortValuesOnly<Integer, W, K>())
- .apply(ParDo.of(
- new ToIsmMetadataRecordForKeyDoFn<K, V, W>(inputCoder.getKeyCoder(), windowCoder)));
- windowMapKeysMetadata.setCoder(ismCoder);
-
- // Set that all these outputs should be materialized using an indexed format.
- runner.addPCollectionRequiringIndexedFormat(perHashWithReifiedWindows);
- runner.addPCollectionRequiringIndexedFormat(windowMapSizeMetadata);
- runner.addPCollectionRequiringIndexedFormat(windowMapKeysMetadata);
-
- PCollectionList<IsmRecord<WindowedValue<V>>> outputs =
- PCollectionList.of(ImmutableList.of(
- perHashWithReifiedWindows, windowMapSizeMetadata, windowMapKeysMetadata));
-
- return Pipeline.applyTransform(outputs,
- Flatten.<IsmRecord<WindowedValue<V>>>pCollections())
- .apply(CreatePCollectionView.<IsmRecord<WindowedValue<V>>,
- ViewT>of(view));
- }
-
- @Override
- protected String getKindString() {
- return "BatchViewAsMultimap";
- }
-
- static <V> IsmRecordCoder<WindowedValue<V>> coderForMapLike(
- Coder<? extends BoundedWindow> windowCoder, Coder<?> keyCoder, Coder<V> valueCoder) {
- // TODO: swap to use a variable length long coder which has values which compare
- // the same as their byte representation compare lexicographically within the key coder
- return IsmRecordCoder.of(
- 1, // We use only the key for hashing when producing value records
- 2, // Since the key is not present, we add the window to the hash when
- // producing metadata records
- ImmutableList.of(
- MetadataKeyCoder.of(keyCoder),
- windowCoder,
- BigEndianLongCoder.of()),
- FullWindowedValueCoder.of(valueCoder, windowCoder));
- }
- }
-
- /**
- * A {@code Map<K, V2>} backed by a {@code Map<K, V1>} and a function that transforms
- * {@code V1 -> V2}.
- */
- static class TransformedMap<K, V1, V2>
- extends ForwardingMap<K, V2> {
- private final Function<V1, V2> transform;
- private final Map<K, V1> originalMap;
- private final Map<K, V2> transformedMap;
-
- private TransformedMap(Function<V1, V2> transform, Map<K, V1> originalMap) {
- this.transform = transform;
- this.originalMap = Collections.unmodifiableMap(originalMap);
- this.transformedMap = Maps.transformValues(originalMap, transform);
- }
-
- @Override
- protected Map<K, V2> delegate() {
- return transformedMap;
- }
- }
-
- /**
- * A {@link Coder} for {@link TransformedMap}s.
- */
- static class TransformedMapCoder<K, V1, V2>
- extends StandardCoder<TransformedMap<K, V1, V2>> {
- private final Coder<Function<V1, V2>> transformCoder;
- private final Coder<Map<K, V1>> originalMapCoder;
-
- private TransformedMapCoder(
- Coder<Function<V1, V2>> transformCoder, Coder<Map<K, V1>> originalMapCoder) {
- this.transformCoder = transformCoder;
- this.originalMapCoder = originalMapCoder;
- }
-
- public static <K, V1, V2> TransformedMapCoder<K, V1, V2> of(
- Coder<Function<V1, V2>> transformCoder, Coder<Map<K, V1>> originalMapCoder) {
- return new TransformedMapCoder<>(transformCoder, originalMapCoder);
- }
-
- @JsonCreator
- public static <K, V1, V2> TransformedMapCoder<K, V1, V2> of(
- @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
- List<Coder<?>> components) {
- checkArgument(components.size() == 2,
- "Expecting 2 components, got " + components.size());
- @SuppressWarnings("unchecked")
- Coder<Function<V1, V2>> transformCoder = (Coder<Function<V1, V2>>) components.get(0);
- @SuppressWarnings("unchecked")
- Coder<Map<K, V1>> originalMapCoder = (Coder<Map<K, V1>>) components.get(1);
- return of(transformCoder, originalMapCoder);
- }
-
- @Override
- public void encode(TransformedMap<K, V1, V2> value, OutputStream outStream,
- Coder.Context context) throws CoderException, IOException {
- transformCoder.encode(value.transform, outStream, context.nested());
- originalMapCoder.encode(value.originalMap, outStream, context.nested());
- }
-
- @Override
- public TransformedMap<K, V1, V2> decode(
- InputStream inStream, Coder.Context context) throws CoderException, IOException {
- return new TransformedMap<>(
- transformCoder.decode(inStream, context.nested()),
- originalMapCoder.decode(inStream, context.nested()));
- }
-
- @Override
- public List<? extends Coder<?>> getCoderArguments() {
<TRUNCATED>
[15/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerHooks.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerHooks.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerHooks.java
new file mode 100644
index 0000000..4d37966
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunnerHooks.java
@@ -0,0 +1,39 @@
+/*
+ * 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.dataflow;
+
+import org.apache.beam.sdk.annotations.Experimental;
+
+import com.google.api.services.dataflow.model.Environment;
+
+/**
+ * An instance of this class can be passed to the
+ * {@link DataflowPipelineRunner} to add user defined hooks to be
+ * invoked at various times during pipeline execution.
+ */
+@Experimental
+public class DataflowPipelineRunnerHooks {
+ /**
+ * Allows the user to modify the environment of their job before their job is submitted
+ * to the service for execution.
+ *
+ * @param environment The environment of the job. Users can make change to this instance in order
+ * to change the environment with which their job executes on the service.
+ */
+ public void modifyEnvironmentBeforeSubmission(Environment environment) {}
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
new file mode 100644
index 0000000..0f2d325
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
@@ -0,0 +1,1059 @@
+/*
+ * 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.dataflow;
+
+import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray;
+import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString;
+import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray;
+import static org.apache.beam.sdk.util.Structs.addBoolean;
+import static org.apache.beam.sdk.util.Structs.addDictionary;
+import static org.apache.beam.sdk.util.Structs.addList;
+import static org.apache.beam.sdk.util.Structs.addLong;
+import static org.apache.beam.sdk.util.Structs.addObject;
+import static org.apache.beam.sdk.util.Structs.addString;
+import static org.apache.beam.sdk.util.Structs.getString;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner.GroupByKeyAndSortValuesOnly;
+import org.apache.beam.runners.dataflow.internal.BigQueryIOTranslator;
+import org.apache.beam.runners.dataflow.internal.PubsubIOTranslator;
+import org.apache.beam.runners.dataflow.internal.ReadTranslator;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.options.StreamingOptions;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Combine;
+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.display.DisplayData;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.AppliedCombineFn;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.DoFnInfo;
+import org.apache.beam.sdk.util.OutputReference;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+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.TupleTag;
+import org.apache.beam.sdk.values.TypedPValue;
+
+import com.google.api.services.dataflow.model.AutoscalingSettings;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.api.services.dataflow.model.Disk;
+import com.google.api.services.dataflow.model.Environment;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.Step;
+import com.google.api.services.dataflow.model.WorkerPool;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * {@link DataflowPipelineTranslator} knows how to translate {@link Pipeline} objects
+ * into Cloud Dataflow Service API {@link Job}s.
+ */
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class DataflowPipelineTranslator {
+ // Must be kept in sync with their internal counterparts.
+ private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineTranslator.class);
+ private static final ObjectMapper MAPPER = new ObjectMapper();
+
+ /**
+ * A map from {@link PTransform} subclass to the corresponding
+ * {@link TransformTranslator} to use to translate that transform.
+ *
+ * <p>A static map that contains system-wide defaults.
+ */
+ private static Map<Class, TransformTranslator> transformTranslators =
+ new HashMap<>();
+
+ /** Provided configuration options. */
+ private final DataflowPipelineOptions options;
+
+ /**
+ * Constructs a translator from the provided options.
+ *
+ * @param options Properties that configure the translator.
+ *
+ * @return The newly created translator.
+ */
+ public static DataflowPipelineTranslator fromOptions(
+ DataflowPipelineOptions options) {
+ return new DataflowPipelineTranslator(options);
+ }
+
+ private DataflowPipelineTranslator(DataflowPipelineOptions options) {
+ this.options = options;
+ }
+
+ /**
+ * Translates a {@link Pipeline} into a {@code JobSpecification}.
+ */
+ public JobSpecification translate(
+ Pipeline pipeline,
+ DataflowPipelineRunner runner,
+ List<DataflowPackage> packages) {
+
+ Translator translator = new Translator(pipeline, runner);
+ Job result = translator.translate(packages);
+ return new JobSpecification(result, Collections.unmodifiableMap(translator.stepNames));
+ }
+
+ /**
+ * The result of a job translation.
+ *
+ * <p>Used to pass the result {@link Job} and any state that was used to construct the job that
+ * may be of use to other classes (eg the {@link PTransform} to StepName mapping).
+ */
+ public static class JobSpecification {
+ private final Job job;
+ private final Map<AppliedPTransform<?, ?, ?>, String> stepNames;
+
+ public JobSpecification(Job job, Map<AppliedPTransform<?, ?, ?>, String> stepNames) {
+ this.job = job;
+ this.stepNames = stepNames;
+ }
+
+ public Job getJob() {
+ return job;
+ }
+
+ /**
+ * Returns the mapping of {@link AppliedPTransform AppliedPTransforms} to the internal step
+ * name for that {@code AppliedPTransform}.
+ */
+ public Map<AppliedPTransform<?, ?, ?>, String> getStepNames() {
+ return stepNames;
+ }
+ }
+
+ /**
+ * Renders a {@link Job} as a string.
+ */
+ public static String jobToString(Job job) {
+ try {
+ return MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(job);
+ } catch (JsonProcessingException exc) {
+ throw new IllegalStateException("Failed to render Job as String.", exc);
+ }
+ }
+
+ /////////////////////////////////////////////////////////////////////////////
+
+ /**
+ * Records that instances of the specified PTransform class
+ * should be translated by default by the corresponding
+ * {@link TransformTranslator}.
+ */
+ public static <TransformT extends PTransform> void registerTransformTranslator(
+ Class<TransformT> transformClass,
+ TransformTranslator<? extends TransformT> transformTranslator) {
+ if (transformTranslators.put(transformClass, transformTranslator) != null) {
+ throw new IllegalArgumentException(
+ "defining multiple translators for " + transformClass);
+ }
+ }
+
+ /**
+ * Returns the {@link TransformTranslator} to use for instances of the
+ * specified PTransform class, or null if none registered.
+ */
+ public <TransformT extends PTransform>
+ TransformTranslator<TransformT> getTransformTranslator(Class<TransformT> transformClass) {
+ return transformTranslators.get(transformClass);
+ }
+
+ /**
+ * A {@link TransformTranslator} knows how to translate
+ * a particular subclass of {@link PTransform} for the
+ * Cloud Dataflow service. It does so by
+ * mutating the {@link TranslationContext}.
+ */
+ public interface TransformTranslator<TransformT extends PTransform> {
+ public void translate(TransformT transform,
+ TranslationContext context);
+ }
+
+ /**
+ * The interface provided to registered callbacks for interacting
+ * with the {@link DataflowPipelineRunner}, including reading and writing the
+ * values of {@link PCollection}s and side inputs ({@link PCollectionView}s).
+ */
+ public interface TranslationContext {
+ /**
+ * Returns the configured pipeline options.
+ */
+ DataflowPipelineOptions getPipelineOptions();
+
+ /**
+ * Returns the input of the currently being translated transform.
+ */
+ <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform);
+
+ /**
+ * Returns the output of the currently being translated transform.
+ */
+ <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform);
+
+ /**
+ * Returns the full name of the currently being translated transform.
+ */
+ String getFullName(PTransform<?, ?> transform);
+
+ /**
+ * Adds a step to the Dataflow workflow for the given transform, with
+ * the given Dataflow step type.
+ * This step becomes "current" for the purpose of {@link #addInput} and
+ * {@link #addOutput}.
+ */
+ public void addStep(PTransform<?, ?> transform, String type);
+
+ /**
+ * Adds a pre-defined step to the Dataflow workflow. The given PTransform should be
+ * consistent with the Step, in terms of input, output and coder types.
+ *
+ * <p>This is a low-level operation, when using this method it is up to
+ * the caller to ensure that names do not collide.
+ */
+ public void addStep(PTransform<?, ? extends PValue> transform, Step step);
+
+ /**
+ * Sets the encoding for the current Dataflow step.
+ */
+ public void addEncodingInput(Coder<?> value);
+
+ /**
+ * Adds an input with the given name and value to the current
+ * Dataflow step.
+ */
+ public void addInput(String name, Boolean value);
+
+ /**
+ * Adds an input with the given name and value to the current
+ * Dataflow step.
+ */
+ public void addInput(String name, String value);
+
+ /**
+ * Adds an input with the given name and value to the current
+ * Dataflow step.
+ */
+ public void addInput(String name, Long value);
+
+ /**
+ * Adds an input with the given name to the previously added Dataflow
+ * step, coming from the specified input PValue.
+ */
+ public void addInput(String name, PInput value);
+
+ /**
+ * Adds an input that is a dictionary of strings to objects.
+ */
+ public void addInput(String name, Map<String, Object> elements);
+
+ /**
+ * Adds an input that is a list of objects.
+ */
+ public void addInput(String name, List<? extends Map<String, Object>> elements);
+
+ /**
+ * Adds an output with the given name to the previously added
+ * Dataflow step, producing the specified output {@code PValue},
+ * including its {@code Coder} if a {@code TypedPValue}. If the
+ * {@code PValue} is a {@code PCollection}, wraps its coder inside
+ * a {@code WindowedValueCoder}.
+ */
+ public void addOutput(String name, PValue value);
+
+ /**
+ * Adds an output with the given name to the previously added
+ * Dataflow step, producing the specified output {@code PValue},
+ * including its {@code Coder} if a {@code TypedPValue}. If the
+ * {@code PValue} is a {@code PCollection}, wraps its coder inside
+ * a {@code ValueOnlyCoder}.
+ */
+ public void addValueOnlyOutput(String name, PValue value);
+
+ /**
+ * Adds an output with the given name to the previously added
+ * CollectionToSingleton Dataflow step, consuming the specified
+ * input {@code PValue} and producing the specified output
+ * {@code PValue}. This step requires special treatment for its
+ * output encoding.
+ */
+ public void addCollectionToSingletonOutput(String name,
+ PValue inputValue,
+ PValue outputValue);
+
+ /**
+ * Encode a PValue reference as an output reference.
+ */
+ public OutputReference asOutputReference(PValue value);
+ }
+
+
+ /////////////////////////////////////////////////////////////////////////////
+
+ /**
+ * Translates a Pipeline into the Dataflow representation.
+ */
+ class Translator implements PipelineVisitor, TranslationContext {
+ /** The Pipeline to translate. */
+ private final Pipeline pipeline;
+
+ /** The runner which will execute the pipeline. */
+ private final DataflowPipelineRunner runner;
+
+ /** The Cloud Dataflow Job representation. */
+ private final Job job = new Job();
+
+ /**
+ * Translator is stateful, as addProperty calls refer to the current step.
+ */
+ private Step currentStep;
+
+ /**
+ * A Map from AppliedPTransform to their unique Dataflow step names.
+ */
+ private final Map<AppliedPTransform<?, ?, ?>, String> stepNames = new HashMap<>();
+
+ /**
+ * A Map from PValues to their output names used by their producer
+ * Dataflow steps.
+ */
+ private final Map<POutput, String> outputNames = new HashMap<>();
+
+ /**
+ * A Map from PValues to the Coders used for them.
+ */
+ private final Map<POutput, Coder<?>> outputCoders = new HashMap<>();
+
+ /**
+ * The transform currently being applied.
+ */
+ private AppliedPTransform<?, ?, ?> currentTransform;
+
+ /**
+ * Constructs a Translator that will translate the specified
+ * Pipeline into Dataflow objects.
+ */
+ public Translator(Pipeline pipeline, DataflowPipelineRunner runner) {
+ this.pipeline = pipeline;
+ this.runner = runner;
+ }
+
+ /**
+ * Translates this Translator's pipeline onto its writer.
+ * @return a Job definition filled in with the type of job, the environment,
+ * and the job steps.
+ */
+ public Job translate(List<DataflowPackage> packages) {
+ job.setName(options.getJobName().toLowerCase());
+
+ Environment environment = new Environment();
+ job.setEnvironment(environment);
+
+ try {
+ environment.setSdkPipelineOptions(
+ MAPPER.readValue(MAPPER.writeValueAsBytes(options), Map.class));
+ } catch (IOException e) {
+ throw new IllegalArgumentException(
+ "PipelineOptions specified failed to serialize to JSON.", e);
+ }
+
+ WorkerPool workerPool = new WorkerPool();
+
+ if (options.getTeardownPolicy() != null) {
+ workerPool.setTeardownPolicy(options.getTeardownPolicy().getTeardownPolicyName());
+ }
+
+ if (options.isStreaming()) {
+ job.setType("JOB_TYPE_STREAMING");
+ } else {
+ job.setType("JOB_TYPE_BATCH");
+ workerPool.setDiskType(options.getWorkerDiskType());
+ }
+
+ if (options.getWorkerMachineType() != null) {
+ workerPool.setMachineType(options.getWorkerMachineType());
+ }
+
+ workerPool.setPackages(packages);
+ workerPool.setNumWorkers(options.getNumWorkers());
+
+ if (options.isStreaming()) {
+ // Use separate data disk for streaming.
+ Disk disk = new Disk();
+ disk.setDiskType(options.getWorkerDiskType());
+ workerPool.setDataDisks(Collections.singletonList(disk));
+ }
+ if (!Strings.isNullOrEmpty(options.getZone())) {
+ workerPool.setZone(options.getZone());
+ }
+ if (!Strings.isNullOrEmpty(options.getNetwork())) {
+ workerPool.setNetwork(options.getNetwork());
+ }
+ if (!Strings.isNullOrEmpty(options.getSubnetwork())) {
+ workerPool.setSubnetwork(options.getSubnetwork());
+ }
+ if (options.getDiskSizeGb() > 0) {
+ workerPool.setDiskSizeGb(options.getDiskSizeGb());
+ }
+ AutoscalingSettings settings = new AutoscalingSettings();
+ if (options.getAutoscalingAlgorithm() != null) {
+ settings.setAlgorithm(options.getAutoscalingAlgorithm().getAlgorithm());
+ }
+ settings.setMaxNumWorkers(options.getMaxNumWorkers());
+ workerPool.setAutoscalingSettings(settings);
+
+ List<WorkerPool> workerPools = new LinkedList<>();
+
+ workerPools.add(workerPool);
+ environment.setWorkerPools(workerPools);
+
+ pipeline.traverseTopologically(this);
+ return job;
+ }
+
+ @Override
+ public DataflowPipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @Override
+ public <InputT extends PInput> InputT getInput(PTransform<InputT, ?> transform) {
+ return (InputT) getCurrentTransform(transform).getInput();
+ }
+
+ @Override
+ public <OutputT extends POutput> OutputT getOutput(PTransform<?, OutputT> transform) {
+ return (OutputT) getCurrentTransform(transform).getOutput();
+ }
+
+ @Override
+ public String getFullName(PTransform<?, ?> transform) {
+ return getCurrentTransform(transform).getFullName();
+ }
+
+ private AppliedPTransform<?, ?, ?> getCurrentTransform(PTransform<?, ?> transform) {
+ checkArgument(
+ currentTransform != null && currentTransform.getTransform() == transform,
+ "can only be called with current transform");
+ return currentTransform;
+ }
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {
+ PTransform<?, ?> transform = node.getTransform();
+ TransformTranslator translator =
+ getTransformTranslator(transform.getClass());
+ if (translator == null) {
+ throw new IllegalStateException(
+ "no translator registered for " + transform);
+ }
+ LOG.debug("Translating {}", transform);
+ currentTransform = AppliedPTransform.of(
+ node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform);
+ translator.translate(transform, this);
+ currentTransform = null;
+ }
+
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ LOG.debug("Checking translation of {}", value);
+ if (value.getProducingTransformInternal() == null) {
+ throw new RuntimeException(
+ "internal error: expecting a PValue "
+ + "to have a producingTransform");
+ }
+ if (!producer.isCompositeNode()) {
+ // Primitive transforms are the only ones assigned step names.
+ asOutputReference(value);
+ }
+ }
+
+ @Override
+ public void addStep(PTransform<?, ?> transform, String type) {
+ String stepName = genStepName();
+ if (stepNames.put(getCurrentTransform(transform), stepName) != null) {
+ throw new IllegalArgumentException(
+ transform + " already has a name specified");
+ }
+ // Start the next "steps" list item.
+ List<Step> steps = job.getSteps();
+ if (steps == null) {
+ steps = new LinkedList<>();
+ job.setSteps(steps);
+ }
+
+ currentStep = new Step();
+ currentStep.setName(stepName);
+ currentStep.setKind(type);
+ steps.add(currentStep);
+ addInput(PropertyNames.USER_NAME, getFullName(transform));
+ addDisplayData(PropertyNames.DISPLAY_DATA, DisplayData.from(transform));
+ }
+
+ @Override
+ public void addStep(PTransform<?, ? extends PValue> transform, Step original) {
+ Step step = original.clone();
+ String stepName = step.getName();
+ if (stepNames.put(getCurrentTransform(transform), stepName) != null) {
+ throw new IllegalArgumentException(transform + " already has a name specified");
+ }
+
+ Map<String, Object> properties = step.getProperties();
+ if (properties != null) {
+ @Nullable List<Map<String, Object>> outputInfoList = null;
+ try {
+ // TODO: This should be done via a Structs accessor.
+ @Nullable List<Map<String, Object>> list =
+ (List<Map<String, Object>>) properties.get(PropertyNames.OUTPUT_INFO);
+ outputInfoList = list;
+ } catch (Exception e) {
+ throw new RuntimeException("Inconsistent dataflow pipeline translation", e);
+ }
+ if (outputInfoList != null && outputInfoList.size() > 0) {
+ Map<String, Object> firstOutputPort = outputInfoList.get(0);
+ @Nullable String name;
+ try {
+ name = getString(firstOutputPort, PropertyNames.OUTPUT_NAME);
+ } catch (Exception e) {
+ name = null;
+ }
+ if (name != null) {
+ registerOutputName(getOutput(transform), name);
+ }
+ }
+ }
+
+ List<Step> steps = job.getSteps();
+ if (steps == null) {
+ steps = new LinkedList<>();
+ job.setSteps(steps);
+ }
+ currentStep = step;
+ steps.add(step);
+ }
+
+ @Override
+ public void addEncodingInput(Coder<?> coder) {
+ CloudObject encoding = SerializableUtils.ensureSerializable(coder);
+ addObject(getProperties(), PropertyNames.ENCODING, encoding);
+ }
+
+ @Override
+ public void addInput(String name, Boolean value) {
+ addBoolean(getProperties(), name, value);
+ }
+
+ @Override
+ public void addInput(String name, String value) {
+ addString(getProperties(), name, value);
+ }
+
+ @Override
+ public void addInput(String name, Long value) {
+ addLong(getProperties(), name, value);
+ }
+
+ @Override
+ public void addInput(String name, Map<String, Object> elements) {
+ addDictionary(getProperties(), name, elements);
+ }
+
+ @Override
+ public void addInput(String name, List<? extends Map<String, Object>> elements) {
+ addList(getProperties(), name, elements);
+ }
+
+ @Override
+ public void addInput(String name, PInput value) {
+ if (value instanceof PValue) {
+ addInput(name, asOutputReference((PValue) value));
+ } else {
+ throw new IllegalStateException("Input must be a PValue");
+ }
+ }
+
+ @Override
+ public void addOutput(String name, PValue value) {
+ Coder<?> coder;
+ if (value instanceof TypedPValue) {
+ coder = ((TypedPValue<?>) value).getCoder();
+ if (value instanceof PCollection) {
+ // Wrap the PCollection element Coder inside a WindowedValueCoder.
+ coder = WindowedValue.getFullCoder(
+ coder,
+ ((PCollection<?>) value).getWindowingStrategy().getWindowFn().windowCoder());
+ }
+ } else {
+ // No output coder to encode.
+ coder = null;
+ }
+ addOutput(name, value, coder);
+ }
+
+ @Override
+ public void addValueOnlyOutput(String name, PValue value) {
+ Coder<?> coder;
+ if (value instanceof TypedPValue) {
+ coder = ((TypedPValue<?>) value).getCoder();
+ if (value instanceof PCollection) {
+ // Wrap the PCollection element Coder inside a ValueOnly
+ // WindowedValueCoder.
+ coder = WindowedValue.getValueOnlyCoder(coder);
+ }
+ } else {
+ // No output coder to encode.
+ coder = null;
+ }
+ addOutput(name, value, coder);
+ }
+
+ @Override
+ public void addCollectionToSingletonOutput(String name,
+ PValue inputValue,
+ PValue outputValue) {
+ Coder<?> inputValueCoder =
+ Preconditions.checkNotNull(outputCoders.get(inputValue));
+ // The inputValueCoder for the input PCollection should be some
+ // WindowedValueCoder of the input PCollection's element
+ // coder.
+ Preconditions.checkState(
+ inputValueCoder instanceof WindowedValue.WindowedValueCoder);
+ // The outputValueCoder for the output should be an
+ // IterableCoder of the inputValueCoder. This is a property
+ // of the backend "CollectionToSingleton" step.
+ Coder<?> outputValueCoder = IterableCoder.of(inputValueCoder);
+ addOutput(name, outputValue, outputValueCoder);
+ }
+
+ /**
+ * Adds an output with the given name to the previously added
+ * Dataflow step, producing the specified output {@code PValue}
+ * with the given {@code Coder} (if not {@code null}).
+ */
+ private void addOutput(String name, PValue value, Coder<?> valueCoder) {
+ registerOutputName(value, name);
+
+ Map<String, Object> properties = getProperties();
+ @Nullable List<Map<String, Object>> outputInfoList = null;
+ try {
+ // TODO: This should be done via a Structs accessor.
+ outputInfoList = (List<Map<String, Object>>) properties.get(PropertyNames.OUTPUT_INFO);
+ } catch (Exception e) {
+ throw new RuntimeException("Inconsistent dataflow pipeline translation", e);
+ }
+ if (outputInfoList == null) {
+ outputInfoList = new ArrayList<>();
+ // TODO: This should be done via a Structs accessor.
+ properties.put(PropertyNames.OUTPUT_INFO, outputInfoList);
+ }
+
+ Map<String, Object> outputInfo = new HashMap<>();
+ addString(outputInfo, PropertyNames.OUTPUT_NAME, name);
+ addString(outputInfo, PropertyNames.USER_NAME, value.getName());
+ if (value instanceof PCollection
+ && runner.doesPCollectionRequireIndexedFormat((PCollection<?>) value)) {
+ addBoolean(outputInfo, PropertyNames.USE_INDEXED_FORMAT, true);
+ }
+ if (valueCoder != null) {
+ // Verify that encoding can be decoded, in order to catch serialization
+ // failures as early as possible.
+ CloudObject encoding = SerializableUtils.ensureSerializable(valueCoder);
+ addObject(outputInfo, PropertyNames.ENCODING, encoding);
+ outputCoders.put(value, valueCoder);
+ }
+
+ outputInfoList.add(outputInfo);
+ }
+
+ private void addDisplayData(String name, DisplayData displayData) {
+ List<Map<String, Object>> list = MAPPER.convertValue(displayData, List.class);
+ addList(getProperties(), name, list);
+ }
+
+ @Override
+ public OutputReference asOutputReference(PValue value) {
+ AppliedPTransform<?, ?, ?> transform =
+ value.getProducingTransformInternal();
+ String stepName = stepNames.get(transform);
+ if (stepName == null) {
+ throw new IllegalArgumentException(transform + " doesn't have a name specified");
+ }
+
+ String outputName = outputNames.get(value);
+ if (outputName == null) {
+ throw new IllegalArgumentException(
+ "output " + value + " doesn't have a name specified");
+ }
+
+ return new OutputReference(stepName, outputName);
+ }
+
+ private Map<String, Object> getProperties() {
+ Map<String, Object> properties = currentStep.getProperties();
+ if (properties == null) {
+ properties = new HashMap<>();
+ currentStep.setProperties(properties);
+ }
+ return properties;
+ }
+
+ /**
+ * Returns a fresh Dataflow step name.
+ */
+ private String genStepName() {
+ return "s" + (stepNames.size() + 1);
+ }
+
+ /**
+ * Records the name of the given output PValue,
+ * within its producing transform.
+ */
+ private void registerOutputName(POutput value, String name) {
+ if (outputNames.put(value, name) != null) {
+ throw new IllegalArgumentException(
+ "output " + value + " already has a name specified");
+ }
+ }
+ }
+
+ /////////////////////////////////////////////////////////////////////////////
+
+ @Override
+ public String toString() {
+ return "DataflowPipelineTranslator#" + hashCode();
+ }
+
+
+ ///////////////////////////////////////////////////////////////////////////
+
+ static {
+ registerTransformTranslator(
+ View.CreatePCollectionView.class,
+ new TransformTranslator<View.CreatePCollectionView>() {
+ @Override
+ public void translate(
+ View.CreatePCollectionView transform,
+ TranslationContext context) {
+ translateTyped(transform, context);
+ }
+
+ private <ElemT, ViewT> void translateTyped(
+ View.CreatePCollectionView<ElemT, ViewT> transform,
+ TranslationContext context) {
+ context.addStep(transform, "CollectionToSingleton");
+ context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
+ context.addCollectionToSingletonOutput(
+ PropertyNames.OUTPUT,
+ context.getInput(transform),
+ context.getOutput(transform));
+ }
+ });
+
+ DataflowPipelineTranslator.registerTransformTranslator(
+ Combine.GroupedValues.class,
+ new DataflowPipelineTranslator.TransformTranslator<Combine.GroupedValues>() {
+ @Override
+ public void translate(
+ Combine.GroupedValues transform,
+ DataflowPipelineTranslator.TranslationContext context) {
+ translateHelper(transform, context);
+ }
+
+ private <K, InputT, OutputT> void translateHelper(
+ final Combine.GroupedValues<K, InputT, OutputT> transform,
+ DataflowPipelineTranslator.TranslationContext context) {
+ context.addStep(transform, "CombineValues");
+ translateInputs(context.getInput(transform), transform.getSideInputs(), context);
+
+ AppliedCombineFn<? super K, ? super InputT, ?, OutputT> fn =
+ transform.getAppliedFn(
+ context.getInput(transform).getPipeline().getCoderRegistry(),
+ context.getInput(transform).getCoder(),
+ context.getInput(transform).getWindowingStrategy());
+
+ context.addEncodingInput(fn.getAccumulatorCoder());
+ context.addInput(
+ PropertyNames.SERIALIZED_FN,
+ byteArrayToJsonString(serializeToByteArray(fn)));
+ context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+ }
+ });
+
+ registerTransformTranslator(
+ Flatten.FlattenPCollectionList.class,
+ new TransformTranslator<Flatten.FlattenPCollectionList>() {
+ @Override
+ public void translate(
+ Flatten.FlattenPCollectionList transform,
+ TranslationContext context) {
+ flattenHelper(transform, context);
+ }
+
+ private <T> void flattenHelper(
+ Flatten.FlattenPCollectionList<T> transform,
+ TranslationContext context) {
+ context.addStep(transform, "Flatten");
+
+ List<OutputReference> inputs = new LinkedList<>();
+ for (PCollection<T> input : context.getInput(transform).getAll()) {
+ inputs.add(context.asOutputReference(input));
+ }
+ context.addInput(PropertyNames.INPUTS, inputs);
+ context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+ }
+ });
+
+ registerTransformTranslator(
+ GroupByKeyAndSortValuesOnly.class,
+ new TransformTranslator<GroupByKeyAndSortValuesOnly>() {
+ @Override
+ public void translate(
+ GroupByKeyAndSortValuesOnly transform,
+ TranslationContext context) {
+ groupByKeyAndSortValuesHelper(transform, context);
+ }
+
+ private <K1, K2, V> void groupByKeyAndSortValuesHelper(
+ GroupByKeyAndSortValuesOnly<K1, K2, V> transform,
+ TranslationContext context) {
+ context.addStep(transform, "GroupByKey");
+ context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
+ context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+ context.addInput(PropertyNames.SORT_VALUES, true);
+
+ // TODO: Add support for combiner lifting once the need arises.
+ context.addInput(
+ PropertyNames.DISALLOW_COMBINER_LIFTING, true);
+ }
+ });
+
+ registerTransformTranslator(
+ GroupByKey.class,
+ new TransformTranslator<GroupByKey>() {
+ @Override
+ public void translate(
+ GroupByKey transform,
+ TranslationContext context) {
+ groupByKeyHelper(transform, context);
+ }
+
+ private <K, V> void groupByKeyHelper(
+ GroupByKey<K, V> transform,
+ TranslationContext context) {
+ context.addStep(transform, "GroupByKey");
+ context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
+ context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+
+ WindowingStrategy<?, ?> windowingStrategy =
+ context.getInput(transform).getWindowingStrategy();
+ boolean isStreaming =
+ context.getPipelineOptions().as(StreamingOptions.class).isStreaming();
+ boolean disallowCombinerLifting =
+ !windowingStrategy.getWindowFn().isNonMerging()
+ || (isStreaming && !transform.fewKeys())
+ // TODO: Allow combiner lifting on the non-default trigger, as appropriate.
+ || !(windowingStrategy.getTrigger().getSpec() instanceof DefaultTrigger);
+ context.addInput(
+ PropertyNames.DISALLOW_COMBINER_LIFTING, disallowCombinerLifting);
+ context.addInput(
+ PropertyNames.SERIALIZED_FN,
+ byteArrayToJsonString(serializeToByteArray(windowingStrategy)));
+ context.addInput(
+ PropertyNames.IS_MERGING_WINDOW_FN,
+ !windowingStrategy.getWindowFn().isNonMerging());
+ }
+ });
+
+ registerTransformTranslator(
+ ParDo.BoundMulti.class,
+ new TransformTranslator<ParDo.BoundMulti>() {
+ @Override
+ public void translate(
+ ParDo.BoundMulti transform,
+ TranslationContext context) {
+ translateMultiHelper(transform, context);
+ }
+
+ private <InputT, OutputT> void translateMultiHelper(
+ ParDo.BoundMulti<InputT, OutputT> transform,
+ TranslationContext context) {
+ context.addStep(transform, "ParallelDo");
+ translateInputs(context.getInput(transform), transform.getSideInputs(), context);
+ translateFn(transform.getFn(), context.getInput(transform).getWindowingStrategy(),
+ transform.getSideInputs(), context.getInput(transform).getCoder(), context);
+ translateOutputs(context.getOutput(transform), context);
+ }
+ });
+
+ registerTransformTranslator(
+ ParDo.Bound.class,
+ new TransformTranslator<ParDo.Bound>() {
+ @Override
+ public void translate(
+ ParDo.Bound transform,
+ TranslationContext context) {
+ translateSingleHelper(transform, context);
+ }
+
+ private <InputT, OutputT> void translateSingleHelper(
+ ParDo.Bound<InputT, OutputT> transform,
+ TranslationContext context) {
+ context.addStep(transform, "ParallelDo");
+ translateInputs(context.getInput(transform), transform.getSideInputs(), context);
+ translateFn(
+ transform.getFn(),
+ context.getInput(transform).getWindowingStrategy(),
+ transform.getSideInputs(), context.getInput(transform).getCoder(), context);
+ context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+ }
+ });
+
+ registerTransformTranslator(
+ Window.Bound.class,
+ new DataflowPipelineTranslator.TransformTranslator<Window.Bound>() {
+ @Override
+ public void translate(
+ Window.Bound transform, TranslationContext context) {
+ translateHelper(transform, context);
+ }
+
+ private <T> void translateHelper(
+ Window.Bound<T> transform, TranslationContext context) {
+ context.addStep(transform, "Bucket");
+ context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
+ context.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+
+ WindowingStrategy<?, ?> strategy = context.getOutput(transform).getWindowingStrategy();
+ byte[] serializedBytes = serializeToByteArray(strategy);
+ String serializedJson = byteArrayToJsonString(serializedBytes);
+ assert Arrays.equals(serializedBytes,
+ jsonStringToByteArray(serializedJson));
+ context.addInput(PropertyNames.SERIALIZED_FN, serializedJson);
+ }
+ });
+
+ ///////////////////////////////////////////////////////////////////////////
+ // IO Translation.
+
+ registerTransformTranslator(
+ BigQueryIO.Read.Bound.class, new BigQueryIOTranslator.ReadTranslator());
+
+ registerTransformTranslator(
+ PubsubIO.Read.Bound.class, new PubsubIOTranslator.ReadTranslator());
+ registerTransformTranslator(
+ DataflowPipelineRunner.StreamingPubsubIOWrite.class,
+ new PubsubIOTranslator.WriteTranslator());
+
+ registerTransformTranslator(Read.Bounded.class, new ReadTranslator());
+ }
+
+ private static void translateInputs(
+ PCollection<?> input,
+ List<PCollectionView<?>> sideInputs,
+ TranslationContext context) {
+ context.addInput(PropertyNames.PARALLEL_INPUT, input);
+ translateSideInputs(sideInputs, context);
+ }
+
+ // Used for ParDo
+ private static void translateSideInputs(
+ List<PCollectionView<?>> sideInputs,
+ TranslationContext context) {
+ Map<String, Object> nonParInputs = new HashMap<>();
+
+ for (PCollectionView<?> view : sideInputs) {
+ nonParInputs.put(
+ view.getTagInternal().getId(),
+ context.asOutputReference(view));
+ }
+
+ context.addInput(PropertyNames.NON_PARALLEL_INPUTS, nonParInputs);
+ }
+
+ private static void translateFn(
+ DoFn fn,
+ WindowingStrategy windowingStrategy,
+ Iterable<PCollectionView<?>> sideInputs,
+ Coder inputCoder,
+ TranslationContext context) {
+ context.addInput(PropertyNames.USER_FN, fn.getClass().getName());
+ context.addInput(
+ PropertyNames.SERIALIZED_FN,
+ byteArrayToJsonString(serializeToByteArray(
+ new DoFnInfo(fn, windowingStrategy, sideInputs, inputCoder))));
+ }
+
+ private static void translateOutputs(
+ PCollectionTuple outputs,
+ TranslationContext context) {
+ for (Map.Entry<TupleTag<?>, PCollection<?>> entry
+ : outputs.getAll().entrySet()) {
+ TupleTag<?> tag = entry.getKey();
+ PCollection<?> output = entry.getValue();
+ context.addOutput(tag.getId(), output);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowServiceException.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowServiceException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowServiceException.java
new file mode 100644
index 0000000..5cd8a4d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowServiceException.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.dataflow;
+
+import javax.annotation.Nullable;
+
+/**
+ * Signals there was an error retrieving information about a job from the Cloud Dataflow Service.
+ */
+public class DataflowServiceException extends DataflowJobException {
+ DataflowServiceException(DataflowPipelineJob job, String message) {
+ this(job, message, null);
+ }
+
+ DataflowServiceException(DataflowPipelineJob job, String message, @Nullable Throwable cause) {
+ super(job, message, cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java
new file mode 100644
index 0000000..1b18c2a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/AssignWindows.java
@@ -0,0 +1,89 @@
+/*
+ * 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.dataflow.internal;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * A primitive {@link PTransform} that implements the {@link Window#into(WindowFn)}
+ * {@link PTransform}.
+ *
+ * For an application of {@link Window#into(WindowFn)} that changes the {@link WindowFn}, applies
+ * a primitive {@link PTransform} in the Dataflow service.
+ *
+ * For an application of {@link Window#into(WindowFn)} that does not change the {@link WindowFn},
+ * applies an identity {@link ParDo} and sets the windowing strategy of the output
+ * {@link PCollection}.
+ *
+ * For internal use only.
+ *
+ * @param <T> the type of input element
+ */
+public class AssignWindows<T> extends PTransform<PCollection<T>, PCollection<T>> {
+ private final Window.Bound<T> transform;
+
+ /**
+ * Builds an instance of this class from the overriden transform.
+ */
+ @SuppressWarnings("unused") // Used via reflection
+ public AssignWindows(Window.Bound<T> transform) {
+ this.transform = transform;
+ }
+
+ @Override
+ public PCollection<T> apply(PCollection<T> input) {
+ WindowingStrategy<?, ?> outputStrategy =
+ transform.getOutputStrategyInternal(input.getWindowingStrategy());
+ if (transform.getWindowFn() != null) {
+ // If the windowFn changed, we create a primitive, and run the AssignWindows operation here.
+ return PCollection.<T>createPrimitiveOutputInternal(
+ input.getPipeline(), outputStrategy, input.isBounded());
+ } else {
+ // If the windowFn didn't change, we just run a pass-through transform and then set the
+ // new windowing strategy.
+ return input.apply(ParDo.named("Identity").of(new DoFn<T, T>() {
+ @Override
+ public void processElement(DoFn<T, T>.ProcessContext c) throws Exception {
+ c.output(c.element());
+ }
+ })).setWindowingStrategyInternal(outputStrategy);
+ }
+ }
+
+ @Override
+ public void validate(PCollection<T> input) {
+ transform.validate(input);
+ }
+
+ @Override
+ protected Coder<?> getDefaultOutputCoder(PCollection<T> input) {
+ return input.getCoder();
+ }
+
+ @Override
+ protected String getKindString() {
+ return "Window.Into()";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/BigQueryIOTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/BigQueryIOTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/BigQueryIOTranslator.java
new file mode 100755
index 0000000..b344f0f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/BigQueryIOTranslator.java
@@ -0,0 +1,72 @@
+/*
+ * 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.dataflow.internal;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.util.PropertyNames;
+
+import com.google.api.services.bigquery.model.TableReference;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * BigQuery transform support code for the Dataflow backend.
+ */
+public class BigQueryIOTranslator {
+ private static final Logger LOG = LoggerFactory.getLogger(BigQueryIOTranslator.class);
+
+ /**
+ * Implements BigQueryIO Read translation for the Dataflow backend.
+ */
+ public static class ReadTranslator
+ implements DataflowPipelineTranslator.TransformTranslator<BigQueryIO.Read.Bound> {
+
+ @Override
+ public void translate(
+ BigQueryIO.Read.Bound transform, DataflowPipelineTranslator.TranslationContext context) {
+ // Actual translation.
+ context.addStep(transform, "ParallelRead");
+ context.addInput(PropertyNames.FORMAT, "bigquery");
+ context.addInput(PropertyNames.BIGQUERY_EXPORT_FORMAT, "FORMAT_AVRO");
+
+ if (transform.getQuery() != null) {
+ context.addInput(PropertyNames.BIGQUERY_QUERY, transform.getQuery());
+ context.addInput(PropertyNames.BIGQUERY_FLATTEN_RESULTS, transform.getFlattenResults());
+ } else {
+ TableReference table = transform.getTable();
+ if (table.getProjectId() == null) {
+ // If user does not specify a project we assume the table to be located in the project
+ // that owns the Dataflow job.
+ String projectIdFromOptions = context.getPipelineOptions().getProject();
+ LOG.warn(String.format(BigQueryIO.SET_PROJECT_FROM_OPTIONS_WARNING, table.getDatasetId(),
+ table.getDatasetId(), table.getTableId(), projectIdFromOptions));
+ table.setProjectId(projectIdFromOptions);
+ }
+
+ context.addInput(PropertyNames.BIGQUERY_TABLE, table.getTableId());
+ context.addInput(PropertyNames.BIGQUERY_DATASET, table.getDatasetId());
+ if (table.getProjectId() != null) {
+ context.addInput(PropertyNames.BIGQUERY_PROJECT, table.getProjectId());
+ }
+ }
+ context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java
new file mode 100755
index 0000000..73e5da0
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java
@@ -0,0 +1,121 @@
+/*
+ * 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.dataflow.internal;
+
+import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray;
+import static org.apache.beam.sdk.util.Structs.addString;
+import static org.apache.beam.sdk.util.Structs.addStringList;
+
+import static com.google.api.client.util.Base64.encodeBase64String;
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.CloudObject;
+
+import com.google.api.services.dataflow.model.SourceMetadata;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ByteString;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+/**
+ * A helper class for supporting sources defined as {@code Source}.
+ *
+ * <p>Provides a bridge between the high-level {@code Source} API and the
+ * low-level {@code CloudSource} class.
+ */
+public class CustomSources {
+ private static final String SERIALIZED_SOURCE = "serialized_source";
+ @VisibleForTesting static final String SERIALIZED_SOURCE_SPLITS = "serialized_source_splits";
+ /**
+ * The current limit on the size of a ReportWorkItemStatus RPC to Google Cloud Dataflow, which
+ * includes the initial splits, is 20 MB.
+ */
+ public static final long DATAFLOW_SPLIT_RESPONSE_API_SIZE_BYTES = 20 * (1 << 20);
+
+ private static final Logger LOG = LoggerFactory.getLogger(CustomSources.class);
+
+ private static final ByteString firstSplitKey = ByteString.copyFromUtf8("0000000000000001");
+
+ public static boolean isFirstUnboundedSourceSplit(ByteString splitKey) {
+ return splitKey.equals(firstSplitKey);
+ }
+
+ private static int getDesiredNumUnboundedSourceSplits(DataflowPipelineOptions options) {
+ if (options.getMaxNumWorkers() > 0) {
+ return options.getMaxNumWorkers();
+ } else if (options.getNumWorkers() > 0) {
+ return options.getNumWorkers() * 3;
+ } else {
+ return 20;
+ }
+ }
+
+ public static com.google.api.services.dataflow.model.Source serializeToCloudSource(
+ Source<?> source, PipelineOptions options) throws Exception {
+ com.google.api.services.dataflow.model.Source cloudSource =
+ new com.google.api.services.dataflow.model.Source();
+ // We ourselves act as the SourceFormat.
+ cloudSource.setSpec(CloudObject.forClass(CustomSources.class));
+ addString(
+ cloudSource.getSpec(), SERIALIZED_SOURCE, encodeBase64String(serializeToByteArray(source)));
+
+ SourceMetadata metadata = new SourceMetadata();
+ if (source instanceof BoundedSource) {
+ BoundedSource<?> boundedSource = (BoundedSource<?>) source;
+ try {
+ metadata.setProducesSortedKeys(boundedSource.producesSortedKeys(options));
+ } catch (Exception e) {
+ LOG.warn("Failed to check if the source produces sorted keys: " + source, e);
+ }
+
+ // Size estimation is best effort so we continue even if it fails here.
+ try {
+ metadata.setEstimatedSizeBytes(boundedSource.getEstimatedSizeBytes(options));
+ } catch (Exception e) {
+ LOG.warn("Size estimation of the source failed: " + source, e);
+ }
+ } else if (source instanceof UnboundedSource) {
+ UnboundedSource<?, ?> unboundedSource = (UnboundedSource<?, ?>) source;
+ metadata.setInfinite(true);
+ List<String> encodedSplits = new ArrayList<>();
+ int desiredNumSplits =
+ getDesiredNumUnboundedSourceSplits(options.as(DataflowPipelineOptions.class));
+ for (UnboundedSource<?, ?> split :
+ unboundedSource.generateInitialSplits(desiredNumSplits, options)) {
+ encodedSplits.add(encodeBase64String(serializeToByteArray(split)));
+ }
+ checkArgument(!encodedSplits.isEmpty(), "UnboundedSources must have at least one split");
+ addStringList(cloudSource.getSpec(), SERIALIZED_SOURCE_SPLITS, encodedSplits);
+ } else {
+ throw new IllegalArgumentException("Unexpected source kind: " + source.getClass());
+ }
+
+ cloudSource.setMetadata(metadata);
+ return cloudSource;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java
new file mode 100755
index 0000000..7a08fde
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java
@@ -0,0 +1,81 @@
+/*
+ * 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.dataflow.internal;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * A mapping relating {@link Aggregator}s and the {@link PTransform} in which they are used.
+ */
+public class DataflowAggregatorTransforms {
+ private final Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorTransforms;
+ private final Multimap<PTransform<?, ?>, AppliedPTransform<?, ?, ?>> transformAppliedTransforms;
+ private final BiMap<AppliedPTransform<?, ?, ?>, String> appliedStepNames;
+
+ public DataflowAggregatorTransforms(
+ Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorTransforms,
+ Map<AppliedPTransform<?, ?, ?>, String> transformStepNames) {
+ this.aggregatorTransforms = aggregatorTransforms;
+ appliedStepNames = HashBiMap.create(transformStepNames);
+
+ transformAppliedTransforms = HashMultimap.create();
+ for (AppliedPTransform<?, ?, ?> appliedTransform : transformStepNames.keySet()) {
+ transformAppliedTransforms.put(appliedTransform.getTransform(), appliedTransform);
+ }
+ }
+
+ /**
+ * Returns true if the provided {@link Aggregator} is used in the constructing {@link Pipeline}.
+ */
+ public boolean contains(Aggregator<?, ?> aggregator) {
+ return aggregatorTransforms.containsKey(aggregator);
+ }
+
+ /**
+ * Gets the step names in which the {@link Aggregator} is used.
+ */
+ public Collection<String> getAggregatorStepNames(Aggregator<?, ?> aggregator) {
+ Collection<String> names = new HashSet<>();
+ Collection<PTransform<?, ?>> transforms = aggregatorTransforms.get(aggregator);
+ for (PTransform<?, ?> transform : transforms) {
+ for (AppliedPTransform<?, ?, ?> applied : transformAppliedTransforms.get(transform)) {
+ names.add(appliedStepNames.get(applied));
+ }
+ }
+ return names;
+ }
+
+ /**
+ * Gets the {@link PTransform} that was assigned the provided step name.
+ */
+ public AppliedPTransform<?, ?, ?> getAppliedTransformForStepName(String stepName) {
+ return appliedStepNames.inverse().get(stepName);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java
new file mode 100755
index 0000000..8ab59fc
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java
@@ -0,0 +1,111 @@
+/*
+ * 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.dataflow.internal;
+
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.PTransform;
+
+import com.google.api.services.dataflow.model.MetricStructuredName;
+import com.google.api.services.dataflow.model.MetricUpdate;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Methods for extracting the values of an {@link Aggregator} from a collection of {@link
+ * MetricUpdate MetricUpdates}.
+ */
+public final class DataflowMetricUpdateExtractor {
+ private static final String STEP_NAME_CONTEXT_KEY = "step";
+ private static final String IS_TENTATIVE_KEY = "tentative";
+
+ private DataflowMetricUpdateExtractor() {
+ // Do not instantiate.
+ }
+
+ /**
+ * Extract the values of the provided {@link Aggregator} at each {@link PTransform} it was used in
+ * according to the provided {@link DataflowAggregatorTransforms} from the given list of {@link
+ * MetricUpdate MetricUpdates}.
+ */
+ public static <OutputT> Map<String, OutputT> fromMetricUpdates(Aggregator<?, OutputT> aggregator,
+ DataflowAggregatorTransforms aggregatorTransforms, List<MetricUpdate> metricUpdates) {
+ Map<String, OutputT> results = new HashMap<>();
+ if (metricUpdates == null) {
+ return results;
+ }
+
+ String aggregatorName = aggregator.getName();
+ Collection<String> aggregatorSteps = aggregatorTransforms.getAggregatorStepNames(aggregator);
+
+ for (MetricUpdate metricUpdate : metricUpdates) {
+ MetricStructuredName metricStructuredName = metricUpdate.getName();
+ Map<String, String> context = metricStructuredName.getContext();
+ if (metricStructuredName.getName().equals(aggregatorName) && context != null
+ && aggregatorSteps.contains(context.get(STEP_NAME_CONTEXT_KEY))) {
+ AppliedPTransform<?, ?, ?> transform =
+ aggregatorTransforms.getAppliedTransformForStepName(
+ context.get(STEP_NAME_CONTEXT_KEY));
+ String fullName = transform.getFullName();
+ // Prefer the tentative (fresher) value if it exists.
+ if (Boolean.parseBoolean(context.get(IS_TENTATIVE_KEY)) || !results.containsKey(fullName)) {
+ results.put(fullName, toValue(aggregator, metricUpdate));
+ }
+ }
+ }
+
+ return results;
+
+ }
+
+ private static <OutputT> OutputT toValue(
+ Aggregator<?, OutputT> aggregator, MetricUpdate metricUpdate) {
+ CombineFn<?, ?, OutputT> combineFn = aggregator.getCombineFn();
+ Class<? super OutputT> outputType = combineFn.getOutputType().getRawType();
+
+ if (outputType.equals(Long.class)) {
+ @SuppressWarnings("unchecked")
+ OutputT asLong = (OutputT) Long.valueOf(toNumber(metricUpdate).longValue());
+ return asLong;
+ }
+ if (outputType.equals(Integer.class)) {
+ @SuppressWarnings("unchecked")
+ OutputT asInt = (OutputT) Integer.valueOf(toNumber(metricUpdate).intValue());
+ return asInt;
+ }
+ if (outputType.equals(Double.class)) {
+ @SuppressWarnings("unchecked")
+ OutputT asDouble = (OutputT) Double.valueOf(toNumber(metricUpdate).doubleValue());
+ return asDouble;
+ }
+ throw new UnsupportedOperationException(
+ "Unsupported Output Type " + outputType + " in aggregator " + aggregator);
+ }
+
+ private static Number toNumber(MetricUpdate update) {
+ if (update.getScalar() instanceof Number) {
+ return (Number) update.getScalar();
+ }
+ throw new IllegalArgumentException(
+ "Metric Update " + update + " does not have a numeric scalar");
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/PubsubIOTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/PubsubIOTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/PubsubIOTranslator.java
new file mode 100755
index 0000000..976f948
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/PubsubIOTranslator.java
@@ -0,0 +1,108 @@
+/*
+ * 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.dataflow.internal;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTranslator;
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext;
+import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * Pubsub transform support code for the Dataflow backend.
+ */
+public class PubsubIOTranslator {
+
+ /**
+ * Implements PubsubIO Read translation for the Dataflow backend.
+ */
+ public static class ReadTranslator<T> implements TransformTranslator<PubsubIO.Read.Bound<T>> {
+ @Override
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ public void translate(
+ PubsubIO.Read.Bound transform,
+ TranslationContext context) {
+ translateReadHelper(transform, context);
+ }
+
+ private <T> void translateReadHelper(
+ PubsubIO.Read.Bound<T> transform,
+ TranslationContext context) {
+ if (!context.getPipelineOptions().isStreaming()) {
+ throw new IllegalArgumentException(
+ "PubsubIO.Read can only be used with the Dataflow streaming runner.");
+ }
+
+ context.addStep(transform, "ParallelRead");
+ context.addInput(PropertyNames.FORMAT, "pubsub");
+ if (transform.getTopic() != null) {
+ context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic().asV1Beta1Path());
+ }
+ if (transform.getSubscription() != null) {
+ context.addInput(
+ PropertyNames.PUBSUB_SUBSCRIPTION, transform.getSubscription().asV1Beta1Path());
+ }
+ if (transform.getTimestampLabel() != null) {
+ context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel());
+ }
+ if (transform.getIdLabel() != null) {
+ context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel());
+ }
+ context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+ }
+ }
+
+ /**
+ * Implements PubsubIO Write translation for the Dataflow backend.
+ */
+ public static class WriteTranslator<T>
+ implements TransformTranslator<DataflowPipelineRunner.StreamingPubsubIOWrite<T>> {
+
+ @Override
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ public void translate(
+ DataflowPipelineRunner.StreamingPubsubIOWrite transform,
+ TranslationContext context) {
+ translateWriteHelper(transform, context);
+ }
+
+ private <T> void translateWriteHelper(
+ DataflowPipelineRunner.StreamingPubsubIOWrite<T> customTransform,
+ TranslationContext context) {
+ if (!context.getPipelineOptions().isStreaming()) {
+ throw new IllegalArgumentException(
+ "PubsubIO.Write is non-primitive for the Dataflow batch runner.");
+ }
+
+ PubsubIO.Write.Bound<T> transform = customTransform.getOverriddenTransform();
+
+ context.addStep(customTransform, "ParallelWrite");
+ context.addInput(PropertyNames.FORMAT, "pubsub");
+ context.addInput(PropertyNames.PUBSUB_TOPIC, transform.getTopic().asV1Beta1Path());
+ if (transform.getTimestampLabel() != null) {
+ context.addInput(PropertyNames.PUBSUB_TIMESTAMP_LABEL, transform.getTimestampLabel());
+ }
+ if (transform.getIdLabel() != null) {
+ context.addInput(PropertyNames.PUBSUB_ID_LABEL, transform.getIdLabel());
+ }
+ context.addEncodingInput(WindowedValue.getValueOnlyCoder(transform.getCoder()));
+ context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(customTransform));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java
new file mode 100755
index 0000000..373738a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java
@@ -0,0 +1,105 @@
+/*
+ * 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.dataflow.internal;
+
+import static org.apache.beam.sdk.util.Structs.addBoolean;
+import static org.apache.beam.sdk.util.Structs.addDictionary;
+import static org.apache.beam.sdk.util.Structs.addLong;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator;
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTranslator;
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext;
+import org.apache.beam.sdk.io.FileBasedSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.Source;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.values.PValue;
+
+import com.google.api.services.dataflow.model.SourceMetadata;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Translator for the {@code Read} {@code PTransform} for the Dataflow back-end.
+ */
+public class ReadTranslator implements TransformTranslator<Read.Bounded<?>> {
+ @Override
+ public void translate(Read.Bounded<?> transform, TranslationContext context) {
+ translateReadHelper(transform.getSource(), transform, context);
+ }
+
+ public static <T> void translateReadHelper(Source<T> source,
+ PTransform<?, ? extends PValue> transform,
+ DataflowPipelineTranslator.TranslationContext context) {
+ try {
+ // TODO: Move this validation out of translation once IOChannelUtils is portable
+ // and can be reconstructed on the worker.
+ if (source instanceof FileBasedSource) {
+ String filePatternOrSpec = ((FileBasedSource<?>) source).getFileOrPatternSpec();
+ context.getPipelineOptions()
+ .getPathValidator()
+ .validateInputFilePatternSupported(filePatternOrSpec);
+ }
+
+ context.addStep(transform, "ParallelRead");
+ context.addInput(PropertyNames.FORMAT, PropertyNames.CUSTOM_SOURCE_FORMAT);
+ context.addInput(
+ PropertyNames.SOURCE_STEP_INPUT,
+ cloudSourceToDictionary(
+ CustomSources.serializeToCloudSource(source, context.getPipelineOptions())));
+ context.addValueOnlyOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ // Represents a cloud Source as a dictionary for encoding inside the {@code SOURCE_STEP_INPUT}
+ // property of CloudWorkflowStep.input.
+ private static Map<String, Object> cloudSourceToDictionary(
+ com.google.api.services.dataflow.model.Source source) {
+ // Do not translate encoding - the source's encoding is translated elsewhere
+ // to the step's output info.
+ Map<String, Object> res = new HashMap<>();
+ addDictionary(res, PropertyNames.SOURCE_SPEC, source.getSpec());
+ if (source.getMetadata() != null) {
+ addDictionary(res, PropertyNames.SOURCE_METADATA,
+ cloudSourceMetadataToDictionary(source.getMetadata()));
+ }
+ if (source.getDoesNotNeedSplitting() != null) {
+ addBoolean(
+ res, PropertyNames.SOURCE_DOES_NOT_NEED_SPLITTING, source.getDoesNotNeedSplitting());
+ }
+ return res;
+ }
+
+ private static Map<String, Object> cloudSourceMetadataToDictionary(SourceMetadata metadata) {
+ Map<String, Object> res = new HashMap<>();
+ if (metadata.getProducesSortedKeys() != null) {
+ addBoolean(res, PropertyNames.SOURCE_PRODUCES_SORTED_KEYS, metadata.getProducesSortedKeys());
+ }
+ if (metadata.getEstimatedSizeBytes() != null) {
+ addLong(res, PropertyNames.SOURCE_ESTIMATED_SIZE_BYTES, metadata.getEstimatedSizeBytes());
+ }
+ if (metadata.getInfinite() != null) {
+ addBoolean(res, PropertyNames.SOURCE_IS_INFINITE, metadata.getInfinite());
+ }
+ return res;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/package-info.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/package-info.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/package-info.java
new file mode 100755
index 0000000..f2e8459
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Implementation of the {@link org.apache.beam.runners.dataflow.DataflowPipelineRunner}.
+ */
+package org.apache.beam.runners.dataflow.internal;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/BlockingDataflowPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/BlockingDataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/BlockingDataflowPipelineOptions.java
new file mode 100644
index 0000000..7fa5ad6
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/BlockingDataflowPipelineOptions.java
@@ -0,0 +1,55 @@
+/*
+ * 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.dataflow.options;
+
+import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Hidden;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+import java.io.PrintStream;
+
+/**
+ * Options that are used to configure the {@link BlockingDataflowPipelineRunner}.
+ */
+@Description("Configure options on the BlockingDataflowPipelineRunner.")
+public interface BlockingDataflowPipelineOptions extends DataflowPipelineOptions {
+ /**
+ * Output stream for job status messages.
+ */
+ @Description("Where messages generated during execution of the Dataflow job will be output.")
+ @JsonIgnore
+ @Hidden
+ @Default.InstanceFactory(StandardOutputFactory.class)
+ PrintStream getJobMessageOutput();
+ void setJobMessageOutput(PrintStream value);
+
+ /**
+ * Returns a default of {@link System#out}.
+ */
+ public static class StandardOutputFactory implements DefaultValueFactory<PrintStream> {
+ @Override
+ public PrintStream create(PipelineOptions options) {
+ return System.out;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java
new file mode 100644
index 0000000..dbfbb16
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.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.dataflow.options;
+
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Hidden;
+
+import com.google.api.services.clouddebugger.v2.model.Debuggee;
+
+import javax.annotation.Nullable;
+
+/**
+ * Options for controlling Cloud Debugger.
+ */
+@Description("[Experimental] Used to configure the Cloud Debugger")
+@Experimental
+@Hidden
+public interface CloudDebuggerOptions {
+
+ /** Whether to enable the Cloud Debugger snapshot agent for the current job. */
+ @Description("Whether to enable the Cloud Debugger snapshot agent for the current job.")
+ boolean getEnableCloudDebugger();
+ void setEnableCloudDebugger(boolean enabled);
+
+ /** The Cloud Debugger debuggee to associate with. This should not be set directly. */
+ @Description("The Cloud Debugger debuggee to associate with. This should not be set directly.")
+ @Hidden
+ @Nullable Debuggee getDebuggee();
+ void setDebuggee(Debuggee debuggee);
+
+ /** The maximum cost (as a ratio of CPU time) allowed for evaluating conditional snapshots. */
+ @Description(
+ "The maximum cost (as a ratio of CPU time) allowed for evaluating conditional snapshots. "
+ + "Should be a double between 0 and 1. "
+ + "Snapshots will be cancelled if evaluating conditions takes more than this ratio of time.")
+ @Default.Double(0.01)
+ double getMaxConditionCost();
+ void setMaxConditionCost(double maxConditionCost);
+}
[19/21] incubator-beam git commit: Increase visibility in PAssert and
ZipFiles utilities
Posted by dh...@apache.org.
Increase visibility in PAssert and ZipFiles utilities
This is needed for package reorganization in runners/google-cloud-dataflow.
Those classes will have to move away from org.apache.beam.sdk.* packages.
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/c08f973c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/c08f973c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/c08f973c
Branch: refs/heads/master
Commit: c08f973cb5585885a2da93c66716dec87670ca30
Parents: 46f7447
Author: Davor Bonaci <da...@google.com>
Authored: Mon Apr 25 14:30:25 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Tue Apr 26 17:59:39 2016 -0700
----------------------------------------------------------------------
.../src/main/java/org/apache/beam/sdk/testing/PAssert.java | 8 ++++----
.../src/main/java/org/apache/beam/sdk/util/ZipFiles.java | 2 +-
2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c08f973c/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
index f328c5b..1265acd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
@@ -98,8 +98,8 @@ public class PAssert {
private static final Logger LOG = LoggerFactory.getLogger(PAssert.class);
- static final String SUCCESS_COUNTER = "PAssertSuccess";
- static final String FAILURE_COUNTER = "PAssertFailure";
+ public static final String SUCCESS_COUNTER = "PAssertSuccess";
+ public static final String FAILURE_COUNTER = "PAssertFailure";
private static int assertCount = 0;
@@ -576,7 +576,7 @@ public class PAssert {
* <p>This is generally useful for assertion functions that
* are serializable but whose underlying data may not have a coder.
*/
- static class OneSideInputAssert<ActualT>
+ public static class OneSideInputAssert<ActualT>
extends PTransform<PBegin, PDone> implements Serializable {
private final transient PTransform<PBegin, PCollectionView<ActualT>> createActual;
private final SerializableFunction<ActualT, Void> checkerFn;
@@ -647,7 +647,7 @@ public class PAssert {
* are not serializable, but have coders (provided
* by the underlying {@link PCollection}s).
*/
- static class TwoSideInputAssert<ActualT, ExpectedT>
+ public static class TwoSideInputAssert<ActualT, ExpectedT>
extends PTransform<PBegin, PDone> implements Serializable {
private final transient PTransform<PBegin, PCollectionView<ActualT>> createActual;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c08f973c/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
index 6d73027..038b9cb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
@@ -226,7 +226,7 @@ public final class ZipFiles {
* @throws IOException the zipping failed, e.g. because the input was not
* readable.
*/
- static void zipDirectory(
+ public static void zipDirectory(
File sourceDirectory,
OutputStream outputStream) throws IOException {
checkNotNull(sourceDirectory);
[02/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java
deleted file mode 100644
index 21bc60e..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/util/PackageUtilTest.java
+++ /dev/null
@@ -1,484 +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.util;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.options.GcsOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.ExpectedLogs;
-import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
-import org.apache.beam.sdk.util.PackageUtil.PackageAttributes;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-
-import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
-import com.google.api.client.googleapis.json.GoogleJsonResponseException;
-import com.google.api.client.http.HttpRequest;
-import com.google.api.client.http.HttpResponse;
-import com.google.api.client.http.HttpStatusCodes;
-import com.google.api.client.http.HttpTransport;
-import com.google.api.client.http.LowLevelHttpRequest;
-import com.google.api.client.json.GenericJson;
-import com.google.api.client.json.Json;
-import com.google.api.client.json.JsonFactory;
-import com.google.api.client.json.jackson2.JacksonFactory;
-import com.google.api.client.testing.http.HttpTesting;
-import com.google.api.client.testing.http.MockHttpTransport;
-import com.google.api.client.testing.http.MockLowLevelHttpRequest;
-import com.google.api.client.testing.http.MockLowLevelHttpResponse;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.io.Files;
-import com.google.common.io.LineReader;
-
-import org.hamcrest.BaseMatcher;
-import org.hamcrest.Description;
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.channels.Channels;
-import java.nio.channels.Pipe;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.regex.Pattern;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
-
-/** Tests for PackageUtil. */
-@RunWith(JUnit4.class)
-public class PackageUtilTest {
- @Rule public ExpectedLogs logged = ExpectedLogs.none(PackageUtil.class);
- @Rule
- public TemporaryFolder tmpFolder = new TemporaryFolder();
-
- @Rule
- public FastNanoClockAndSleeper fastNanoClockAndSleeper = new FastNanoClockAndSleeper();
-
- @Mock
- GcsUtil mockGcsUtil;
-
- // 128 bits, base64 encoded is 171 bits, rounds to 22 bytes
- private static final String HASH_PATTERN = "[a-zA-Z0-9+-]{22}";
-
- // Hamcrest matcher to assert a string matches a pattern
- private static class RegexMatcher extends BaseMatcher<String> {
- private final Pattern pattern;
-
- public RegexMatcher(String regex) {
- this.pattern = Pattern.compile(regex);
- }
-
- @Override
- public boolean matches(Object o) {
- if (!(o instanceof String)) {
- return false;
- }
- return pattern.matcher((String) o).matches();
- }
-
- @Override
- public void describeTo(Description description) {
- description.appendText(String.format("matches regular expression %s", pattern));
- }
-
- public static RegexMatcher matches(String regex) {
- return new RegexMatcher(regex);
- }
- }
-
- @Before
- public void setUp() {
- MockitoAnnotations.initMocks(this);
-
- GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class);
- pipelineOptions.setGcsUtil(mockGcsUtil);
-
- IOChannelUtils.registerStandardIOFactories(pipelineOptions);
- }
-
- private File makeFileWithContents(String name, String contents) throws Exception {
- File tmpFile = tmpFolder.newFile(name);
- Files.write(contents, tmpFile, StandardCharsets.UTF_8);
- tmpFile.setLastModified(0); // required for determinism with directories
- return tmpFile;
- }
-
- static final String STAGING_PATH = GcsPath.fromComponents("somebucket", "base/path").toString();
- private static PackageAttributes makePackageAttributes(File file, String overridePackageName) {
- return PackageUtil.createPackageAttributes(file, STAGING_PATH, overridePackageName);
- }
-
- @Test
- public void testFileWithExtensionPackageNamingAndSize() throws Exception {
- String contents = "This is a test!";
- File tmpFile = makeFileWithContents("file.txt", contents);
- PackageAttributes attr = makePackageAttributes(tmpFile, null);
- DataflowPackage target = attr.getDataflowPackage();
-
- assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN + ".txt"));
- assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
- assertThat(attr.getSize(), equalTo((long) contents.length()));
- }
-
- @Test
- public void testPackageNamingWithFileNoExtension() throws Exception {
- File tmpFile = makeFileWithContents("file", "This is a test!");
- DataflowPackage target = makePackageAttributes(tmpFile, null).getDataflowPackage();
-
- assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN));
- assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
- }
-
- @Test
- public void testPackageNamingWithDirectory() throws Exception {
- File tmpDirectory = tmpFolder.newFolder("folder");
- DataflowPackage target = makePackageAttributes(tmpDirectory, null).getDataflowPackage();
-
- assertThat(target.getName(), RegexMatcher.matches("folder-" + HASH_PATTERN + ".jar"));
- assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
- }
-
- @Test
- public void testPackageNamingWithFilesHavingSameContentsAndSameNames() throws Exception {
- File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
- makeFileWithContents("folder1/folderA/sameName", "This is a test!");
- DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
-
- File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
- makeFileWithContents("folder2/folderA/sameName", "This is a test!");
- DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
-
- assertEquals(target1.getName(), target2.getName());
- assertEquals(target1.getLocation(), target2.getLocation());
- }
-
- @Test
- public void testPackageNamingWithFilesHavingSameContentsButDifferentNames() throws Exception {
- File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
- makeFileWithContents("folder1/folderA/uniqueName1", "This is a test!");
- DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
-
- File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
- makeFileWithContents("folder2/folderA/uniqueName2", "This is a test!");
- DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
-
- assertNotEquals(target1.getName(), target2.getName());
- assertNotEquals(target1.getLocation(), target2.getLocation());
- }
-
- @Test
- public void testPackageNamingWithDirectoriesHavingSameContentsButDifferentNames()
- throws Exception {
- File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
- tmpFolder.newFolder("folder1", "folderA", "uniqueName1");
- DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
-
- File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
- tmpFolder.newFolder("folder2", "folderA", "uniqueName2");
- DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
-
- assertNotEquals(target1.getName(), target2.getName());
- assertNotEquals(target1.getLocation(), target2.getLocation());
- }
-
- @Test
- public void testPackageUploadWithLargeClasspathLogsWarning() throws Exception {
- File tmpFile = makeFileWithContents("file.txt", "This is a test!");
- // all files will be present and cached so no upload needed.
- when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(tmpFile.length());
-
- List<String> classpathElements = Lists.newLinkedList();
- for (int i = 0; i < 1005; ++i) {
- String eltName = "element" + i;
- classpathElements.add(eltName + '=' + tmpFile.getAbsolutePath());
- }
-
- PackageUtil.stageClasspathElements(classpathElements, STAGING_PATH);
-
- logged.verifyWarn("Your classpath contains 1005 elements, which Google Cloud Dataflow");
- }
-
- @Test
- public void testPackageUploadWithFileSucceeds() throws Exception {
- Pipe pipe = Pipe.open();
- String contents = "This is a test!";
- File tmpFile = makeFileWithContents("file.txt", contents);
- when(mockGcsUtil.fileSize(any(GcsPath.class)))
- .thenThrow(new FileNotFoundException("some/path"));
- when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
-
- List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
- ImmutableList.of(tmpFile.getAbsolutePath()), STAGING_PATH);
- DataflowPackage target = Iterables.getOnlyElement(targets);
-
- verify(mockGcsUtil).fileSize(any(GcsPath.class));
- verify(mockGcsUtil).create(any(GcsPath.class), anyString());
- verifyNoMoreInteractions(mockGcsUtil);
-
- assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN + ".txt"));
- assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
- assertThat(new LineReader(Channels.newReader(pipe.source(), "UTF-8")).readLine(),
- equalTo(contents));
- }
-
- @Test
- public void testPackageUploadWithDirectorySucceeds() throws Exception {
- Pipe pipe = Pipe.open();
- File tmpDirectory = tmpFolder.newFolder("folder");
- tmpFolder.newFolder("folder", "empty_directory");
- tmpFolder.newFolder("folder", "directory");
- makeFileWithContents("folder/file.txt", "This is a test!");
- makeFileWithContents("folder/directory/file.txt", "This is also a test!");
-
- when(mockGcsUtil.fileSize(any(GcsPath.class)))
- .thenThrow(new FileNotFoundException("some/path"));
- when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
-
- PackageUtil.stageClasspathElements(
- ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
-
- verify(mockGcsUtil).fileSize(any(GcsPath.class));
- verify(mockGcsUtil).create(any(GcsPath.class), anyString());
- verifyNoMoreInteractions(mockGcsUtil);
-
- ZipInputStream inputStream = new ZipInputStream(Channels.newInputStream(pipe.source()));
- List<String> zipEntryNames = new ArrayList<>();
- for (ZipEntry entry = inputStream.getNextEntry(); entry != null;
- entry = inputStream.getNextEntry()) {
- zipEntryNames.add(entry.getName());
- }
-
- assertThat(zipEntryNames,
- containsInAnyOrder("directory/file.txt", "empty_directory/", "file.txt"));
- }
-
- @Test
- public void testPackageUploadWithEmptyDirectorySucceeds() throws Exception {
- Pipe pipe = Pipe.open();
- File tmpDirectory = tmpFolder.newFolder("folder");
-
- when(mockGcsUtil.fileSize(any(GcsPath.class)))
- .thenThrow(new FileNotFoundException("some/path"));
- when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
-
- List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
- ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
- DataflowPackage target = Iterables.getOnlyElement(targets);
-
- verify(mockGcsUtil).fileSize(any(GcsPath.class));
- verify(mockGcsUtil).create(any(GcsPath.class), anyString());
- verifyNoMoreInteractions(mockGcsUtil);
-
- assertThat(target.getName(), RegexMatcher.matches("folder-" + HASH_PATTERN + ".jar"));
- assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
- assertNull(new ZipInputStream(Channels.newInputStream(pipe.source())).getNextEntry());
- }
-
- @Test(expected = RuntimeException.class)
- public void testPackageUploadFailsWhenIOExceptionThrown() throws Exception {
- File tmpFile = makeFileWithContents("file.txt", "This is a test!");
- when(mockGcsUtil.fileSize(any(GcsPath.class)))
- .thenThrow(new FileNotFoundException("some/path"));
- when(mockGcsUtil.create(any(GcsPath.class), anyString()))
- .thenThrow(new IOException("Fake Exception: Upload error"));
-
- try {
- PackageUtil.stageClasspathElements(
- ImmutableList.of(tmpFile.getAbsolutePath()),
- STAGING_PATH, fastNanoClockAndSleeper);
- } finally {
- verify(mockGcsUtil).fileSize(any(GcsPath.class));
- verify(mockGcsUtil, times(5)).create(any(GcsPath.class), anyString());
- verifyNoMoreInteractions(mockGcsUtil);
- }
- }
-
- @Test
- public void testPackageUploadFailsWithPermissionsErrorGivesDetailedMessage() throws Exception {
- File tmpFile = makeFileWithContents("file.txt", "This is a test!");
- when(mockGcsUtil.fileSize(any(GcsPath.class)))
- .thenThrow(new FileNotFoundException("some/path"));
- when(mockGcsUtil.create(any(GcsPath.class), anyString()))
- .thenThrow(new IOException("Failed to write to GCS path " + STAGING_PATH,
- googleJsonResponseException(
- HttpStatusCodes.STATUS_CODE_FORBIDDEN, "Permission denied", "Test message")));
-
- try {
- PackageUtil.stageClasspathElements(
- ImmutableList.of(tmpFile.getAbsolutePath()),
- STAGING_PATH, fastNanoClockAndSleeper);
- fail("Expected RuntimeException");
- } catch (RuntimeException e) {
- assertTrue("Expected IOException containing detailed message.",
- e.getCause() instanceof IOException);
- assertThat(e.getCause().getMessage(),
- Matchers.allOf(
- Matchers.containsString("Uploaded failed due to permissions error"),
- Matchers.containsString(
- "Stale credentials can be resolved by executing 'gcloud auth login'")));
- } finally {
- verify(mockGcsUtil).fileSize(any(GcsPath.class));
- verify(mockGcsUtil).create(any(GcsPath.class), anyString());
- verifyNoMoreInteractions(mockGcsUtil);
- }
- }
-
- @Test
- public void testPackageUploadEventuallySucceeds() throws Exception {
- Pipe pipe = Pipe.open();
- File tmpFile = makeFileWithContents("file.txt", "This is a test!");
- when(mockGcsUtil.fileSize(any(GcsPath.class)))
- .thenThrow(new FileNotFoundException("some/path"));
- when(mockGcsUtil.create(any(GcsPath.class), anyString()))
- .thenThrow(new IOException("Fake Exception: 410 Gone")) // First attempt fails
- .thenReturn(pipe.sink()); // second attempt succeeds
-
- try {
- PackageUtil.stageClasspathElements(
- ImmutableList.of(tmpFile.getAbsolutePath()),
- STAGING_PATH,
- fastNanoClockAndSleeper);
- } finally {
- verify(mockGcsUtil).fileSize(any(GcsPath.class));
- verify(mockGcsUtil, times(2)).create(any(GcsPath.class), anyString());
- verifyNoMoreInteractions(mockGcsUtil);
- }
- }
-
- @Test
- public void testPackageUploadIsSkippedWhenFileAlreadyExists() throws Exception {
- File tmpFile = makeFileWithContents("file.txt", "This is a test!");
- when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(tmpFile.length());
-
- PackageUtil.stageClasspathElements(
- ImmutableList.of(tmpFile.getAbsolutePath()), STAGING_PATH);
-
- verify(mockGcsUtil).fileSize(any(GcsPath.class));
- verifyNoMoreInteractions(mockGcsUtil);
- }
-
- @Test
- public void testPackageUploadIsNotSkippedWhenSizesAreDifferent() throws Exception {
- Pipe pipe = Pipe.open();
- File tmpDirectory = tmpFolder.newFolder("folder");
- tmpFolder.newFolder("folder", "empty_directory");
- tmpFolder.newFolder("folder", "directory");
- makeFileWithContents("folder/file.txt", "This is a test!");
- makeFileWithContents("folder/directory/file.txt", "This is also a test!");
- when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(Long.MAX_VALUE);
- when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
-
- PackageUtil.stageClasspathElements(
- ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
-
- verify(mockGcsUtil).fileSize(any(GcsPath.class));
- verify(mockGcsUtil).create(any(GcsPath.class), anyString());
- verifyNoMoreInteractions(mockGcsUtil);
- }
-
- @Test
- public void testPackageUploadWithExplicitPackageName() throws Exception {
- Pipe pipe = Pipe.open();
- File tmpFile = makeFileWithContents("file.txt", "This is a test!");
- final String overriddenName = "alias.txt";
-
- when(mockGcsUtil.fileSize(any(GcsPath.class)))
- .thenThrow(new FileNotFoundException("some/path"));
- when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
-
- List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
- ImmutableList.of(overriddenName + "=" + tmpFile.getAbsolutePath()), STAGING_PATH);
- DataflowPackage target = Iterables.getOnlyElement(targets);
-
- verify(mockGcsUtil).fileSize(any(GcsPath.class));
- verify(mockGcsUtil).create(any(GcsPath.class), anyString());
- verifyNoMoreInteractions(mockGcsUtil);
-
- assertThat(target.getName(), equalTo(overriddenName));
- assertThat(target.getLocation(),
- RegexMatcher.matches(STAGING_PATH + "/file-" + HASH_PATTERN + ".txt"));
- }
-
- @Test
- public void testPackageUploadIsSkippedWithNonExistentResource() throws Exception {
- String nonExistentFile =
- IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "non-existent-file");
- assertEquals(Collections.EMPTY_LIST, PackageUtil.stageClasspathElements(
- ImmutableList.of(nonExistentFile), STAGING_PATH));
- }
-
- /**
- * Builds a fake GoogleJsonResponseException for testing API error handling.
- */
- private static GoogleJsonResponseException googleJsonResponseException(
- final int status, final String reason, final String message) throws IOException {
- final JsonFactory jsonFactory = new JacksonFactory();
- HttpTransport transport = new MockHttpTransport() {
- @Override
- public LowLevelHttpRequest buildRequest(String method, String url) throws IOException {
- ErrorInfo errorInfo = new ErrorInfo();
- errorInfo.setReason(reason);
- errorInfo.setMessage(message);
- errorInfo.setFactory(jsonFactory);
- GenericJson error = new GenericJson();
- error.set("code", status);
- error.set("errors", Arrays.asList(errorInfo));
- error.setFactory(jsonFactory);
- GenericJson errorResponse = new GenericJson();
- errorResponse.set("error", error);
- errorResponse.setFactory(jsonFactory);
- return new MockLowLevelHttpRequest().setResponse(
- new MockLowLevelHttpResponse().setContent(errorResponse.toPrettyString())
- .setContentType(Json.MEDIA_TYPE).setStatusCode(status));
- }
- };
- HttpRequest request =
- transport.createRequestFactory().buildGetRequest(HttpTesting.SIMPLE_GENERIC_URL);
- request.setThrowExceptionOnExecuteError(false);
- HttpResponse response = request.execute();
- return GoogleJsonResponseException.from(jsonFactory, response);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
----------------------------------------------------------------------
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 cdd3b84..a51a24e 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
@@ -56,7 +56,7 @@ import javax.annotation.Nullable;
* <li>System property "beamTestPipelineOptions" must contain a JSON delimited list of pipeline
* options. For example:
* <pre>{@code [
- * "--runner=org.apache.beam.sdk.testing.TestDataflowPipelineRunner",
+ * "--runner=org.apache.beam.runners.dataflow.testing.TestDataflowPipelineRunner",
* "--project=mygcpproject",
* "--stagingLocation=gs://mygcsbucket/path"
* ]}</pre>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
index 4b2bb7b..3e4fc86 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/MinimalWordCount.java
@@ -17,11 +17,11 @@
*/
package ${package};
+import org.apache.beam.runners.dataflow.BlockingDataflowPipelineRunner;
+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.DataflowPipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.BlockingDataflowPipelineRunner;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ParDo;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
index f997521..7dea9fe 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WindowedWordCount.java
@@ -17,11 +17,8 @@
*/
package ${package};
-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 ${package}.common.DataflowExampleUtils;
+
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.BigQueryIO;
@@ -37,6 +34,11 @@ import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
+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 org.joda.time.Duration;
import org.joda.time.Instant;
import org.slf4j.Logger;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
index 8c89283..fc1f4b5 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
@@ -17,9 +17,9 @@
*/
package ${package};
+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.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.DefaultValueFactory;
import org.apache.beam.sdk.options.Description;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleOptions.java
index 9e7136d..e3bf7c5 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleOptions.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleOptions.java
@@ -17,7 +17,7 @@
*/
package ${package}.common;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
index 0ad449a..4914d4c 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java
@@ -17,6 +17,11 @@
*/
package ${package}.common;
+import org.apache.beam.runners.dataflow.DataflowPipelineJob;
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+
import com.google.api.client.googleapis.json.GoogleJsonResponseException;
import com.google.api.client.googleapis.services.AbstractGoogleClientRequest;
import com.google.api.services.bigquery.Bigquery;
@@ -34,12 +39,8 @@ import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.options.BigQueryOptions;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.runners.DataflowPipelineJob;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
import org.apache.beam.sdk.runners.DirectPipelineRunner;
import org.apache.beam.sdk.transforms.IntraBundleParallelization;
-import org.apache.beam.sdk.util.MonitoringUtil;
import org.apache.beam.sdk.util.Transport;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
index 9e65c4f..279f2e0 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExampleBigQueryTableOptions.java
@@ -17,13 +17,14 @@
*/
package ${package}.common;
-import com.google.api.services.bigquery.model.TableSchema;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.DefaultValueFactory;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
+import com.google.api.services.bigquery.model.TableSchema;
+
/**
* Options that can be used to configure BigQuery tables in Dataflow examples.
* The project defaults to the project being used to run the example.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
index b5acf69..8a7c9cf 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/ExamplePubsubTopicOptions.java
@@ -17,7 +17,7 @@
*/
package ${package}.common;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.DefaultValueFactory;
import org.apache.beam.sdk.options.Description;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java
index d8bab1f..5c182b2 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java
@@ -17,12 +17,9 @@
*/
package ${package}.common;
-import com.google.api.services.pubsub.Pubsub;
-import com.google.api.services.pubsub.model.PublishRequest;
-import com.google.api.services.pubsub.model.PubsubMessage;
+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.DataflowPipelineOptions;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
@@ -30,6 +27,10 @@ import org.apache.beam.sdk.options.Validation;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.IntraBundleParallelization;
import org.apache.beam.sdk.util.Transport;
+
+import com.google.api.services.pubsub.Pubsub;
+import com.google.api.services.pubsub.model.PublishRequest;
+import com.google.api.services.pubsub.model.PubsubMessage;
import com.google.common.collect.ImmutableMap;
import java.io.IOException;
[07/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
new file mode 100644
index 0000000..3a39e41
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
@@ -0,0 +1,967 @@
+/*
+ * 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.dataflow;
+
+import static org.apache.beam.sdk.util.Structs.addObject;
+import static org.apache.beam.sdk.util.Structs.getDictionary;
+import static org.apache.beam.sdk.util.Structs.getString;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.hasKey;
+import static org.hamcrest.core.IsInstanceOf.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.RecordingPipelineVisitor;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.OutputReference;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.Structs;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.TupleTag;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.Step;
+import com.google.api.services.dataflow.model.WorkerPool;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+
+import org.hamcrest.Matchers;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.internal.matchers.ThrowableMessageMatcher;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.ArgumentMatcher;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests for DataflowPipelineTranslator.
+ */
+@RunWith(JUnit4.class)
+public class DataflowPipelineTranslatorTest implements Serializable {
+
+ @Rule public transient ExpectedException thrown = ExpectedException.none();
+
+ // A Custom Mockito matcher for an initial Job that checks that all
+ // expected fields are set.
+ private static class IsValidCreateRequest extends ArgumentMatcher<Job> {
+ @Override
+ public boolean matches(Object o) {
+ Job job = (Job) o;
+ return job.getId() == null
+ && job.getProjectId() == null
+ && job.getName() != null
+ && job.getType() != null
+ && job.getEnvironment() != null
+ && job.getSteps() != null
+ && job.getCurrentState() == null
+ && job.getCurrentStateTime() == null
+ && job.getExecutionInfo() == null
+ && job.getCreateTime() == null;
+ }
+ }
+
+ private Pipeline buildPipeline(DataflowPipelineOptions options) {
+ options.setRunner(DataflowPipelineRunner.class);
+ Pipeline p = Pipeline.create(options);
+
+ p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
+ .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
+
+ return p;
+ }
+
+ private static Dataflow buildMockDataflow(
+ ArgumentMatcher<Job> jobMatcher) throws IOException {
+ Dataflow mockDataflowClient = mock(Dataflow.class);
+ Dataflow.Projects mockProjects = mock(Dataflow.Projects.class);
+ Dataflow.Projects.Jobs mockJobs = mock(Dataflow.Projects.Jobs.class);
+ Dataflow.Projects.Jobs.Create mockRequest = mock(
+ Dataflow.Projects.Jobs.Create.class);
+
+ when(mockDataflowClient.projects()).thenReturn(mockProjects);
+ when(mockProjects.jobs()).thenReturn(mockJobs);
+ when(mockJobs.create(eq("someProject"), argThat(jobMatcher)))
+ .thenReturn(mockRequest);
+
+ Job resultJob = new Job();
+ resultJob.setId("newid");
+ when(mockRequest.execute()).thenReturn(resultJob);
+ return mockDataflowClient;
+ }
+
+ private static DataflowPipelineOptions buildPipelineOptions() throws IOException {
+ GcsUtil mockGcsUtil = mock(GcsUtil.class);
+ when(mockGcsUtil.expand(any(GcsPath.class))).then(new Answer<List<GcsPath>>() {
+ @Override
+ public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
+ return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
+ }
+ });
+ when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(true);
+ when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
+
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setGcpCredential(new TestCredential());
+ options.setJobName("some-job-name");
+ options.setProject("some-project");
+ options.setTempLocation(GcsPath.fromComponents("somebucket", "some/path").toString());
+ options.setFilesToStage(new LinkedList<String>());
+ options.setDataflowClient(buildMockDataflow(new IsValidCreateRequest()));
+ options.setGcsUtil(mockGcsUtil);
+ return options;
+ }
+
+ @Test
+ public void testSettingOfSdkPipelineOptions() throws IOException {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setRunner(DataflowPipelineRunner.class);
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ // Note that the contents of this materialized map may be changed by the act of reading an
+ // option, which will cause the default to get materialized whereas it would otherwise be
+ // left absent. It is permissible to simply alter this test to reflect current behavior.
+ Map<String, Object> settings = new HashMap<>();
+ settings.put("appName", "DataflowPipelineTranslatorTest");
+ settings.put("project", "some-project");
+ settings.put("pathValidatorClass",
+ "org.apache.beam.runners.dataflow.util.DataflowPathValidator");
+ settings.put("runner", "org.apache.beam.runners.dataflow.DataflowPipelineRunner");
+ settings.put("jobName", "some-job-name");
+ settings.put("tempLocation", "gs://somebucket/some/path");
+ settings.put("stagingLocation", "gs://somebucket/some/path/staging");
+ settings.put("stableUniqueNames", "WARNING");
+ settings.put("streaming", false);
+ settings.put("numberOfWorkerHarnessThreads", 0);
+ settings.put("experiments", null);
+
+ Map<String, Object> sdkPipelineOptions = job.getEnvironment().getSdkPipelineOptions();
+ assertThat(sdkPipelineOptions, hasKey("options"));
+ assertEquals(settings, sdkPipelineOptions.get("options"));
+ }
+
+ @Test
+ public void testNetworkConfig() throws IOException {
+ final String testNetwork = "test-network";
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setNetwork(testNetwork);
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(1, job.getEnvironment().getWorkerPools().size());
+ assertEquals(testNetwork,
+ job.getEnvironment().getWorkerPools().get(0).getNetwork());
+ }
+
+ @Test
+ public void testNetworkConfigMissing() throws IOException {
+ DataflowPipelineOptions options = buildPipelineOptions();
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(1, job.getEnvironment().getWorkerPools().size());
+ assertNull(job.getEnvironment().getWorkerPools().get(0).getNetwork());
+ }
+
+ @Test
+ public void testSubnetworkConfig() throws IOException {
+ final String testSubnetwork = "regions/REGION/subnetworks/SUBNETWORK";
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setSubnetwork(testSubnetwork);
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(1, job.getEnvironment().getWorkerPools().size());
+ assertEquals(testSubnetwork,
+ job.getEnvironment().getWorkerPools().get(0).getSubnetwork());
+ }
+
+ @Test
+ public void testSubnetworkConfigMissing() throws IOException {
+ DataflowPipelineOptions options = buildPipelineOptions();
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(1, job.getEnvironment().getWorkerPools().size());
+ assertNull(job.getEnvironment().getWorkerPools().get(0).getSubnetwork());
+ }
+
+ @Test
+ public void testScalingAlgorithmMissing() throws IOException {
+ DataflowPipelineOptions options = buildPipelineOptions();
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(1, job.getEnvironment().getWorkerPools().size());
+ // Autoscaling settings are always set.
+ assertNull(
+ job
+ .getEnvironment()
+ .getWorkerPools()
+ .get(0)
+ .getAutoscalingSettings()
+ .getAlgorithm());
+ assertEquals(
+ 0,
+ job
+ .getEnvironment()
+ .getWorkerPools()
+ .get(0)
+ .getAutoscalingSettings()
+ .getMaxNumWorkers()
+ .intValue());
+ }
+
+ @Test
+ public void testScalingAlgorithmNone() throws IOException {
+ final DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType noScaling =
+ DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType.NONE;
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setAutoscalingAlgorithm(noScaling);
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(1, job.getEnvironment().getWorkerPools().size());
+ assertEquals(
+ "AUTOSCALING_ALGORITHM_NONE",
+ job
+ .getEnvironment()
+ .getWorkerPools()
+ .get(0)
+ .getAutoscalingSettings()
+ .getAlgorithm());
+ assertEquals(
+ 0,
+ job
+ .getEnvironment()
+ .getWorkerPools()
+ .get(0)
+ .getAutoscalingSettings()
+ .getMaxNumWorkers()
+ .intValue());
+ }
+
+ @Test
+ public void testMaxNumWorkersIsPassedWhenNoAlgorithmIsSet() throws IOException {
+ final DataflowPipelineWorkerPoolOptions.AutoscalingAlgorithmType noScaling = null;
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setMaxNumWorkers(42);
+ options.setAutoscalingAlgorithm(noScaling);
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(1, job.getEnvironment().getWorkerPools().size());
+ assertNull(
+ job
+ .getEnvironment()
+ .getWorkerPools()
+ .get(0)
+ .getAutoscalingSettings()
+ .getAlgorithm());
+ assertEquals(
+ 42,
+ job
+ .getEnvironment()
+ .getWorkerPools()
+ .get(0)
+ .getAutoscalingSettings()
+ .getMaxNumWorkers()
+ .intValue());
+ }
+
+ @Test
+ public void testZoneConfig() throws IOException {
+ final String testZone = "test-zone-1";
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setZone(testZone);
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(1, job.getEnvironment().getWorkerPools().size());
+ assertEquals(testZone,
+ job.getEnvironment().getWorkerPools().get(0).getZone());
+ }
+
+ @Test
+ public void testWorkerMachineTypeConfig() throws IOException {
+ final String testMachineType = "test-machine-type";
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setWorkerMachineType(testMachineType);
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(1, job.getEnvironment().getWorkerPools().size());
+
+ WorkerPool workerPool = job.getEnvironment().getWorkerPools().get(0);
+ assertEquals(testMachineType, workerPool.getMachineType());
+ }
+
+ @Test
+ public void testDiskSizeGbConfig() throws IOException {
+ final Integer diskSizeGb = 1234;
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setDiskSizeGb(diskSizeGb);
+
+ Pipeline p = buildPipeline(options);
+ p.traverseTopologically(new RecordingPipelineVisitor());
+ Job job =
+ DataflowPipelineTranslator.fromOptions(options)
+ .translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(1, job.getEnvironment().getWorkerPools().size());
+ assertEquals(diskSizeGb,
+ job.getEnvironment().getWorkerPools().get(0).getDiskSizeGb());
+ }
+
+ @Test
+ public void testPredefinedAddStep() throws Exception {
+ DataflowPipelineOptions options = buildPipelineOptions();
+
+ DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+ DataflowPipelineTranslator.registerTransformTranslator(
+ EmbeddedTransform.class, new EmbeddedTranslator());
+
+ // Create a predefined step using another pipeline
+ Step predefinedStep = createPredefinedStep();
+
+ // Create a pipeline that the predefined step will be embedded into
+ Pipeline pipeline = Pipeline.create(options);
+ pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in"))
+ .apply(ParDo.of(new NoOpFn()))
+ .apply(new EmbeddedTransform(predefinedStep.clone()))
+ .apply(ParDo.of(new NoOpFn()));
+ Job job =
+ translator
+ .translate(
+ pipeline,
+ (DataflowPipelineRunner) pipeline.getRunner(),
+ Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ List<Step> steps = job.getSteps();
+ assertEquals(4, steps.size());
+
+ // The input to the embedded step should match the output of the step before
+ Map<String, Object> step1Out = getOutputPortReference(steps.get(1));
+ Map<String, Object> step2In = getDictionary(
+ steps.get(2).getProperties(), PropertyNames.PARALLEL_INPUT);
+ assertEquals(step1Out, step2In);
+
+ // The output from the embedded step should match the input of the step after
+ Map<String, Object> step2Out = getOutputPortReference(steps.get(2));
+ Map<String, Object> step3In = getDictionary(
+ steps.get(3).getProperties(), PropertyNames.PARALLEL_INPUT);
+ assertEquals(step2Out, step3In);
+
+ // The step should not have been modified other than remapping the input
+ Step predefinedStepClone = predefinedStep.clone();
+ Step embeddedStepClone = steps.get(2).clone();
+ predefinedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT);
+ embeddedStepClone.getProperties().remove(PropertyNames.PARALLEL_INPUT);
+ assertEquals(predefinedStepClone, embeddedStepClone);
+ }
+
+ /**
+ * Construct a OutputReference for the output of the step.
+ */
+ private static OutputReference getOutputPortReference(Step step) throws Exception {
+ // TODO: This should be done via a Structs accessor.
+ @SuppressWarnings("unchecked")
+ List<Map<String, Object>> output =
+ (List<Map<String, Object>>) step.getProperties().get(PropertyNames.OUTPUT_INFO);
+ String outputTagId = getString(Iterables.getOnlyElement(output), PropertyNames.OUTPUT_NAME);
+ return new OutputReference(step.getName(), outputTagId);
+ }
+
+ /**
+ * Returns a Step for a DoFn by creating and translating a pipeline.
+ */
+ private static Step createPredefinedStep() throws Exception {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+ Pipeline pipeline = Pipeline.create(options);
+ String stepName = "DoFn1";
+ pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in"))
+ .apply(ParDo.of(new NoOpFn()).named(stepName))
+ .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/out"));
+ Job job =
+ translator
+ .translate(
+ pipeline,
+ (DataflowPipelineRunner) pipeline.getRunner(),
+ Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ assertEquals(13, job.getSteps().size());
+ Step step = job.getSteps().get(1);
+ assertEquals(stepName, getString(step.getProperties(), PropertyNames.USER_NAME));
+ return step;
+ }
+
+ private static class NoOpFn extends DoFn<String, String> {
+ @Override public void processElement(ProcessContext c) throws Exception {
+ c.output(c.element());
+ }
+ }
+
+ /**
+ * A placeholder transform that will be used to substitute a predefined Step.
+ */
+ private static class EmbeddedTransform
+ extends PTransform<PCollection<String>, PCollection<String>> {
+ private final Step step;
+
+ public EmbeddedTransform(Step step) {
+ this.step = step;
+ }
+
+ @Override
+ public PCollection<String> apply(PCollection<String> input) {
+ return PCollection.createPrimitiveOutputInternal(
+ input.getPipeline(),
+ WindowingStrategy.globalDefault(),
+ input.isBounded());
+ }
+
+ @Override
+ protected Coder<?> getDefaultOutputCoder() {
+ return StringUtf8Coder.of();
+ }
+ }
+
+ /**
+ * A TransformTranslator that adds the predefined Step using
+ * {@link TranslationContext#addStep} and remaps the input port reference.
+ */
+ private static class EmbeddedTranslator
+ implements DataflowPipelineTranslator.TransformTranslator<EmbeddedTransform> {
+ @Override public void translate(EmbeddedTransform transform, TranslationContext context) {
+ addObject(transform.step.getProperties(), PropertyNames.PARALLEL_INPUT,
+ context.asOutputReference(context.getInput(transform)));
+ context.addStep(transform, transform.step);
+ }
+ }
+
+ /**
+ * A composite transform that returns an output that is unrelated to
+ * the input.
+ */
+ private static class UnrelatedOutputCreator
+ extends PTransform<PCollection<Integer>, PCollection<Integer>> {
+
+ @Override
+ public PCollection<Integer> apply(PCollection<Integer> input) {
+ // Apply an operation so that this is a composite transform.
+ input.apply(Count.<Integer>perElement());
+
+ // Return a value unrelated to the input.
+ return input.getPipeline().apply(Create.of(1, 2, 3, 4));
+ }
+
+ @Override
+ protected Coder<?> getDefaultOutputCoder() {
+ return VarIntCoder.of();
+ }
+ }
+
+ /**
+ * A composite transform that returns an output that is unbound.
+ */
+ private static class UnboundOutputCreator
+ extends PTransform<PCollection<Integer>, PDone> {
+
+ @Override
+ public PDone apply(PCollection<Integer> input) {
+ // Apply an operation so that this is a composite transform.
+ input.apply(Count.<Integer>perElement());
+
+ return PDone.in(input.getPipeline());
+ }
+
+ @Override
+ protected Coder<?> getDefaultOutputCoder() {
+ return VoidCoder.of();
+ }
+ }
+
+ /**
+ * A composite transform that returns a partially bound output.
+ *
+ * <p>This is not allowed and will result in a failure.
+ */
+ private static class PartiallyBoundOutputCreator
+ extends PTransform<PCollection<Integer>, PCollectionTuple> {
+
+ public final TupleTag<Integer> sumTag = new TupleTag<>("sum");
+ public final TupleTag<Void> doneTag = new TupleTag<>("done");
+
+ @Override
+ public PCollectionTuple apply(PCollection<Integer> input) {
+ PCollection<Integer> sum = input.apply(Sum.integersGlobally());
+
+ // Fails here when attempting to construct a tuple with an unbound object.
+ return PCollectionTuple.of(sumTag, sum)
+ .and(doneTag, PCollection.<Void>createPrimitiveOutputInternal(
+ input.getPipeline(),
+ WindowingStrategy.globalDefault(),
+ input.isBounded()));
+ }
+ }
+
+ @Test
+ public void testMultiGraphPipelineSerialization() throws IOException {
+ Pipeline p = Pipeline.create(buildPipelineOptions());
+
+ PCollection<Integer> input = p.begin()
+ .apply(Create.of(1, 2, 3));
+
+ input.apply(new UnrelatedOutputCreator());
+ input.apply(new UnboundOutputCreator());
+
+ DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(
+ PipelineOptionsFactory.as(DataflowPipelineOptions.class));
+
+ // Check that translation doesn't fail.
+ t.translate(
+ p, (DataflowPipelineRunner) p.getRunner(), Collections.<DataflowPackage>emptyList());
+ }
+
+ @Test
+ public void testPartiallyBoundFailure() throws IOException {
+ Pipeline p = Pipeline.create(buildPipelineOptions());
+
+ PCollection<Integer> input = p.begin()
+ .apply(Create.of(1, 2, 3));
+
+ thrown.expect(IllegalStateException.class);
+ input.apply(new PartiallyBoundOutputCreator());
+
+ Assert.fail("Failure expected from use of partially bound output");
+ }
+
+ /**
+ * This tests a few corner cases that should not crash.
+ */
+ @Test
+ public void testGoodWildcards() throws Exception {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ Pipeline pipeline = Pipeline.create(options);
+ DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options);
+
+ applyRead(pipeline, "gs://bucket/foo");
+ applyRead(pipeline, "gs://bucket/foo/");
+ applyRead(pipeline, "gs://bucket/foo/*");
+ applyRead(pipeline, "gs://bucket/foo/?");
+ applyRead(pipeline, "gs://bucket/foo/[0-9]");
+ applyRead(pipeline, "gs://bucket/foo/*baz*");
+ applyRead(pipeline, "gs://bucket/foo/*baz?");
+ applyRead(pipeline, "gs://bucket/foo/[0-9]baz?");
+ applyRead(pipeline, "gs://bucket/foo/baz/*");
+ applyRead(pipeline, "gs://bucket/foo/baz/*wonka*");
+ applyRead(pipeline, "gs://bucket/foo/*baz/wonka*");
+ applyRead(pipeline, "gs://bucket/foo*/baz");
+ applyRead(pipeline, "gs://bucket/foo?/baz");
+ applyRead(pipeline, "gs://bucket/foo[0-9]/baz");
+
+ // Check that translation doesn't fail.
+ t.translate(
+ pipeline,
+ (DataflowPipelineRunner) pipeline.getRunner(),
+ Collections.<DataflowPackage>emptyList());
+ }
+
+ private void applyRead(Pipeline pipeline, String path) {
+ pipeline.apply("Read(" + path + ")", TextIO.Read.from(path));
+ }
+
+ /**
+ * Recursive wildcards are not supported.
+ * This tests "**".
+ */
+ @Test
+ public void testBadWildcardRecursive() throws Exception {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ Pipeline pipeline = Pipeline.create(options);
+ DataflowPipelineTranslator t = DataflowPipelineTranslator.fromOptions(options);
+
+ pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz"));
+
+ // Check that translation does fail.
+ thrown.expectCause(Matchers.allOf(
+ instanceOf(IllegalArgumentException.class),
+ ThrowableMessageMatcher.hasMessage(containsString("Unsupported wildcard usage"))));
+ t.translate(
+ pipeline,
+ (DataflowPipelineRunner) pipeline.getRunner(),
+ Collections.<DataflowPackage>emptyList());
+ }
+
+ @Test
+ public void testToSingletonTranslation() throws Exception {
+ // A "change detector" test that makes sure the translation
+ // of getting a PCollectionView<T> does not change
+ // in bad ways during refactor
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setExperiments(ImmutableList.of("disable_ism_side_input"));
+ DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+
+ Pipeline pipeline = Pipeline.create(options);
+ pipeline.apply(Create.of(1))
+ .apply(View.<Integer>asSingleton());
+ Job job =
+ translator
+ .translate(
+ pipeline,
+ (DataflowPipelineRunner) pipeline.getRunner(),
+ Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ List<Step> steps = job.getSteps();
+ assertEquals(2, steps.size());
+
+ Step createStep = steps.get(0);
+ assertEquals("ParallelRead", createStep.getKind());
+
+ Step collectionToSingletonStep = steps.get(1);
+ assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
+
+ }
+
+ @Test
+ public void testToIterableTranslation() throws Exception {
+ // A "change detector" test that makes sure the translation
+ // of getting a PCollectionView<Iterable<T>> does not change
+ // in bad ways during refactor
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ options.setExperiments(ImmutableList.of("disable_ism_side_input"));
+ DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+
+ Pipeline pipeline = Pipeline.create(options);
+ pipeline.apply(Create.of(1, 2, 3))
+ .apply(View.<Integer>asIterable());
+ Job job =
+ translator
+ .translate(
+ pipeline,
+ (DataflowPipelineRunner) pipeline.getRunner(),
+ Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ List<Step> steps = job.getSteps();
+ assertEquals(2, steps.size());
+
+ Step createStep = steps.get(0);
+ assertEquals("ParallelRead", createStep.getKind());
+
+ Step collectionToSingletonStep = steps.get(1);
+ assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
+ }
+
+ @Test
+ public void testToSingletonTranslationWithIsmSideInput() throws Exception {
+ // A "change detector" test that makes sure the translation
+ // of getting a PCollectionView<T> does not change
+ // in bad ways during refactor
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+
+ Pipeline pipeline = Pipeline.create(options);
+ pipeline.apply(Create.of(1))
+ .apply(View.<Integer>asSingleton());
+ Job job =
+ translator
+ .translate(
+ pipeline,
+ (DataflowPipelineRunner) pipeline.getRunner(),
+ Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ List<Step> steps = job.getSteps();
+ assertEquals(5, steps.size());
+
+ @SuppressWarnings("unchecked")
+ List<Map<String, Object>> toIsmRecordOutputs =
+ (List<Map<String, Object>>) steps.get(3).getProperties().get(PropertyNames.OUTPUT_INFO);
+ assertTrue(
+ Structs.getBoolean(Iterables.getOnlyElement(toIsmRecordOutputs), "use_indexed_format"));
+
+ Step collectionToSingletonStep = steps.get(4);
+ assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
+ }
+
+ @Test
+ public void testToIterableTranslationWithIsmSideInput() throws Exception {
+ // A "change detector" test that makes sure the translation
+ // of getting a PCollectionView<Iterable<T>> does not change
+ // in bad ways during refactor
+
+ DataflowPipelineOptions options = buildPipelineOptions();
+ DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+
+ Pipeline pipeline = Pipeline.create(options);
+ pipeline.apply(Create.of(1, 2, 3))
+ .apply(View.<Integer>asIterable());
+ Job job =
+ translator
+ .translate(
+ pipeline,
+ (DataflowPipelineRunner) pipeline.getRunner(),
+ Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ List<Step> steps = job.getSteps();
+ assertEquals(3, steps.size());
+
+ @SuppressWarnings("unchecked")
+ List<Map<String, Object>> toIsmRecordOutputs =
+ (List<Map<String, Object>>) steps.get(1).getProperties().get(PropertyNames.OUTPUT_INFO);
+ assertTrue(
+ Structs.getBoolean(Iterables.getOnlyElement(toIsmRecordOutputs), "use_indexed_format"));
+
+
+ Step collectionToSingletonStep = steps.get(2);
+ assertEquals("CollectionToSingleton", collectionToSingletonStep.getKind());
+ }
+
+ @Test
+ public void testStepDisplayData() throws Exception {
+ DataflowPipelineOptions options = buildPipelineOptions();
+ DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options);
+ Pipeline pipeline = Pipeline.create(options);
+
+ DoFn<Integer, Integer> fn1 = new DoFn<Integer, Integer>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ c.output(c.element());
+ }
+
+ @Override
+ public void populateDisplayData(DisplayData.Builder builder) {
+ builder
+ .add("foo", "bar")
+ .add("foo2", DataflowPipelineTranslatorTest.class)
+ .withLabel("Test Class")
+ .withLinkUrl("http://www.google.com");
+ }
+ };
+
+ DoFn<Integer, Integer> fn2 = new DoFn<Integer, Integer>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ c.output(c.element());
+ }
+
+ @Override
+ public void populateDisplayData(DisplayData.Builder builder) {
+ builder.add("foo3", 1234);
+ }
+ };
+
+ ParDo.Bound<Integer, Integer> parDo1 = ParDo.of(fn1);
+ ParDo.Bound<Integer, Integer> parDo2 = ParDo.of(fn2);
+ pipeline
+ .apply(Create.of(1, 2, 3))
+ .apply(parDo1)
+ .apply(parDo2);
+
+ Job job =
+ translator
+ .translate(
+ pipeline,
+ (DataflowPipelineRunner) pipeline.getRunner(),
+ Collections.<DataflowPackage>emptyList())
+ .getJob();
+
+ List<Step> steps = job.getSteps();
+ assertEquals(3, steps.size());
+
+ Map<String, Object> parDo1Properties = steps.get(1).getProperties();
+ Map<String, Object> parDo2Properties = steps.get(2).getProperties();
+ assertThat(parDo1Properties, hasKey("display_data"));
+
+ Collection<Map<String, String>> fn1displayData =
+ (Collection<Map<String, String>>) parDo1Properties.get("display_data");
+ Collection<Map<String, String>> fn2displayData =
+ (Collection<Map<String, String>>) parDo2Properties.get("display_data");
+
+ ImmutableSet<ImmutableMap<String, Object>> expectedFn1DisplayData = ImmutableSet.of(
+ ImmutableMap.<String, Object>builder()
+ .put("key", "foo")
+ .put("type", "STRING")
+ .put("value", "bar")
+ .put("namespace", fn1.getClass().getName())
+ .build(),
+ ImmutableMap.<String, Object>builder()
+ .put("key", "fn")
+ .put("type", "JAVA_CLASS")
+ .put("value", fn1.getClass().getName())
+ .put("shortValue", fn1.getClass().getSimpleName())
+ .put("namespace", parDo1.getClass().getName())
+ .build(),
+ ImmutableMap.<String, Object>builder()
+ .put("key", "foo2")
+ .put("type", "JAVA_CLASS")
+ .put("value", DataflowPipelineTranslatorTest.class.getName())
+ .put("shortValue", DataflowPipelineTranslatorTest.class.getSimpleName())
+ .put("namespace", fn1.getClass().getName())
+ .put("label", "Test Class")
+ .put("linkUrl", "http://www.google.com")
+ .build()
+ );
+
+ ImmutableSet<ImmutableMap<String, Object>> expectedFn2DisplayData = ImmutableSet.of(
+ ImmutableMap.<String, Object>builder()
+ .put("key", "fn")
+ .put("type", "JAVA_CLASS")
+ .put("value", fn2.getClass().getName())
+ .put("shortValue", fn2.getClass().getSimpleName())
+ .put("namespace", parDo2.getClass().getName())
+ .build(),
+ ImmutableMap.<String, Object>builder()
+ .put("key", "foo3")
+ .put("type", "INTEGER")
+ .put("value", 1234L)
+ .put("namespace", fn2.getClass().getName())
+ .build()
+ );
+
+ assertEquals(expectedFn1DisplayData, ImmutableSet.copyOf(fn1displayData));
+ assertEquals(expectedFn2DisplayData, ImmutableSet.copyOf(fn2displayData));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/internal/CustomSourcesTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/internal/CustomSourcesTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/internal/CustomSourcesTest.java
new file mode 100644
index 0000000..ed86be2
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/internal/CustomSourcesTest.java
@@ -0,0 +1,276 @@
+/*
+ * 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.dataflow.internal;
+import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Sample;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.common.base.Preconditions;
+
+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.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link CustomSources}.
+ */
+@RunWith(JUnit4.class)
+public class CustomSourcesTest {
+ @Rule public ExpectedException expectedException = ExpectedException.none();
+ @Rule public ExpectedLogs logged = ExpectedLogs.none(CustomSources.class);
+
+ static class TestIO {
+ public static Read fromRange(int from, int to) {
+ return new Read(from, to, false);
+ }
+
+ static class Read extends BoundedSource<Integer> {
+ final int from;
+ final int to;
+ final boolean produceTimestamps;
+
+ Read(int from, int to, boolean produceTimestamps) {
+ this.from = from;
+ this.to = to;
+ this.produceTimestamps = produceTimestamps;
+ }
+
+ public Read withTimestampsMillis() {
+ return new Read(from, to, true);
+ }
+
+ @Override
+ public List<Read> splitIntoBundles(long desiredBundleSizeBytes, PipelineOptions options)
+ throws Exception {
+ List<Read> res = new ArrayList<>();
+ DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
+ float step = 1.0f * (to - from) / dataflowOptions.getNumWorkers();
+ for (int i = 0; i < dataflowOptions.getNumWorkers(); ++i) {
+ res.add(new Read(
+ Math.round(from + i * step), Math.round(from + (i + 1) * step),
+ produceTimestamps));
+ }
+ return res;
+ }
+
+ @Override
+ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+ return 8 * (to - from);
+ }
+
+ @Override
+ public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+ return true;
+ }
+
+ @Override
+ public BoundedReader<Integer> createReader(PipelineOptions options) throws IOException {
+ return new RangeReader(this);
+ }
+
+ @Override
+ public void validate() {}
+
+ @Override
+ public String toString() {
+ return "[" + from + ", " + to + ")";
+ }
+
+ @Override
+ public Coder<Integer> getDefaultOutputCoder() {
+ return BigEndianIntegerCoder.of();
+ }
+
+ private static class RangeReader extends BoundedReader<Integer> {
+ // To verify that BasicSerializableSourceFormat calls our methods according to protocol.
+ enum State {
+ UNSTARTED,
+ STARTED,
+ FINISHED
+ }
+ private Read source;
+ private int current = -1;
+ private State state = State.UNSTARTED;
+
+ public RangeReader(Read source) {
+ this.source = source;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ Preconditions.checkState(state == State.UNSTARTED);
+ state = State.STARTED;
+ current = source.from;
+ return (current < source.to);
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ Preconditions.checkState(state == State.STARTED);
+ if (current == source.to - 1) {
+ state = State.FINISHED;
+ return false;
+ }
+ current++;
+ return true;
+ }
+
+ @Override
+ public Integer getCurrent() {
+ Preconditions.checkState(state == State.STARTED);
+ return current;
+ }
+
+ @Override
+ public Instant getCurrentTimestamp() {
+ return source.produceTimestamps
+ ? new Instant(current /* as millis */) : BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+
+ @Override
+ public void close() throws IOException {
+ Preconditions.checkState(state == State.STARTED || state == State.FINISHED);
+ state = State.FINISHED;
+ }
+
+ @Override
+ public Read getCurrentSource() {
+ return source;
+ }
+
+ @Override
+ public Read splitAtFraction(double fraction) {
+ int proposedIndex = (int) (source.from + fraction * (source.to - source.from));
+ if (proposedIndex <= current) {
+ return null;
+ }
+ Read primary = new Read(source.from, proposedIndex, source.produceTimestamps);
+ Read residual = new Read(proposedIndex, source.to, source.produceTimestamps);
+ this.source = primary;
+ return residual;
+ }
+
+ @Override
+ public Double getFractionConsumed() {
+ return (current == -1)
+ ? 0.0
+ : (1.0 * (1 + current - source.from) / (source.to - source.from));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testDirectPipelineWithoutTimestamps() throws Exception {
+ Pipeline p = TestPipeline.create();
+ PCollection<Integer> sum = p
+ .apply(Read.from(TestIO.fromRange(10, 20)))
+ .apply(Sum.integersGlobally())
+ .apply(Sample.<Integer>any(1));
+
+ PAssert.thatSingleton(sum).isEqualTo(145);
+ p.run();
+ }
+
+ @Test
+ public void testDirectPipelineWithTimestamps() throws Exception {
+ Pipeline p = TestPipeline.create();
+ PCollection<Integer> sums =
+ p.apply(Read.from(TestIO.fromRange(10, 20).withTimestampsMillis()))
+ .apply(Window.<Integer>into(FixedWindows.of(Duration.millis(3))))
+ .apply(Sum.integersGlobally().withoutDefaults());
+ // Should group into [10 11] [12 13 14] [15 16 17] [18 19].
+ PAssert.that(sums).containsInAnyOrder(21, 37, 39, 48);
+ p.run();
+ }
+
+ @Test
+ public void testRangeProgressAndSplitAtFraction() throws Exception {
+ // Show basic usage of getFractionConsumed and splitAtFraction.
+ // This test only tests TestIO itself, not BasicSerializableSourceFormat.
+
+ DataflowPipelineOptions options =
+ PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
+ TestIO.Read source = TestIO.fromRange(10, 20);
+ try (BoundedSource.BoundedReader<Integer> reader = source.createReader(options)) {
+ assertEquals(0, reader.getFractionConsumed().intValue());
+ assertTrue(reader.start());
+ assertEquals(0.1, reader.getFractionConsumed(), 1e-6);
+ assertTrue(reader.advance());
+ assertEquals(0.2, reader.getFractionConsumed(), 1e-6);
+ // Already past 0.0 and 0.1.
+ assertNull(reader.splitAtFraction(0.0));
+ assertNull(reader.splitAtFraction(0.1));
+
+ {
+ TestIO.Read residual = (TestIO.Read) reader.splitAtFraction(0.5);
+ assertNotNull(residual);
+ TestIO.Read primary = (TestIO.Read) reader.getCurrentSource();
+ assertThat(readFromSource(primary, options), contains(10, 11, 12, 13, 14));
+ assertThat(readFromSource(residual, options), contains(15, 16, 17, 18, 19));
+ }
+
+ // Range is now [10, 15) and we are at 12.
+ {
+ TestIO.Read residual = (TestIO.Read) reader.splitAtFraction(0.8); // give up 14.
+ assertNotNull(residual);
+ TestIO.Read primary = (TestIO.Read) reader.getCurrentSource();
+ assertThat(readFromSource(primary, options), contains(10, 11, 12, 13));
+ assertThat(readFromSource(residual, options), contains(14));
+ }
+
+ assertTrue(reader.advance());
+ assertEquals(12, reader.getCurrent().intValue());
+ assertTrue(reader.advance());
+ assertEquals(13, reader.getCurrent().intValue());
+ assertFalse(reader.advance());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
new file mode 100644
index 0000000..bfc99e8
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.dataflow.io;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.testing.TestDataflowPipelineOptions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+
+import com.google.common.collect.ImmutableList;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.List;
+
+/**
+ * {@link DataflowPipelineRunner} specific tests for TextIO Read and Write transforms.
+ */
+@RunWith(JUnit4.class)
+public class DataflowTextIOTest {
+
+ private TestDataflowPipelineOptions buildTestPipelineOptions() {
+ TestDataflowPipelineOptions options =
+ PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
+ options.setGcpCredential(new TestCredential());
+ return options;
+ }
+
+ private GcsUtil buildMockGcsUtil() throws IOException {
+ GcsUtil mockGcsUtil = Mockito.mock(GcsUtil.class);
+
+ // Any request to open gets a new bogus channel
+ Mockito
+ .when(mockGcsUtil.open(Mockito.any(GcsPath.class)))
+ .then(new Answer<SeekableByteChannel>() {
+ @Override
+ public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
+ return FileChannel.open(
+ Files.createTempFile("channel-", ".tmp"),
+ StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
+ }
+ });
+
+ // Any request for expansion returns a list containing the original GcsPath
+ // This is required to pass validation that occurs in TextIO during apply()
+ Mockito
+ .when(mockGcsUtil.expand(Mockito.any(GcsPath.class)))
+ .then(new Answer<List<GcsPath>>() {
+ @Override
+ public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
+ return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
+ }
+ });
+
+ return mockGcsUtil;
+ }
+
+ /**
+ * This tests a few corner cases that should not crash.
+ */
+ @Test
+ public void testGoodWildcards() throws Exception {
+ TestDataflowPipelineOptions options = buildTestPipelineOptions();
+ options.setGcsUtil(buildMockGcsUtil());
+
+ Pipeline pipeline = Pipeline.create(options);
+
+ applyRead(pipeline, "gs://bucket/foo");
+ applyRead(pipeline, "gs://bucket/foo/");
+ applyRead(pipeline, "gs://bucket/foo/*");
+ applyRead(pipeline, "gs://bucket/foo/?");
+ applyRead(pipeline, "gs://bucket/foo/[0-9]");
+ applyRead(pipeline, "gs://bucket/foo/*baz*");
+ applyRead(pipeline, "gs://bucket/foo/*baz?");
+ applyRead(pipeline, "gs://bucket/foo/[0-9]baz?");
+ applyRead(pipeline, "gs://bucket/foo/baz/*");
+ applyRead(pipeline, "gs://bucket/foo/baz/*wonka*");
+ applyRead(pipeline, "gs://bucket/foo/*baz/wonka*");
+ applyRead(pipeline, "gs://bucket/foo*/baz");
+ applyRead(pipeline, "gs://bucket/foo?/baz");
+ applyRead(pipeline, "gs://bucket/foo[0-9]/baz");
+
+ // Check that running doesn't fail.
+ pipeline.run();
+ }
+
+ private void applyRead(Pipeline pipeline, String path) {
+ pipeline.apply("Read(" + path + ")", TextIO.Read.from(path));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java
new file mode 100644
index 0000000..71b6b57
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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.dataflow.options;
+
+import static org.hamcrest.Matchers.hasEntry;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link DataflowPipelineDebugOptions}. */
+@RunWith(JUnit4.class)
+public class DataflowPipelineDebugOptionsTest {
+ @Test
+ public void testTransformNameMapping() throws Exception {
+ DataflowPipelineDebugOptions options = PipelineOptionsFactory
+ .fromArgs(new String[]{"--transformNameMapping={\"a\":\"b\",\"foo\":\"\",\"bar\":\"baz\"}"})
+ .as(DataflowPipelineDebugOptions.class);
+ assertEquals(3, options.getTransformNameMapping().size());
+ assertThat(options.getTransformNameMapping(), hasEntry("a", "b"));
+ assertThat(options.getTransformNameMapping(), hasEntry("foo", ""));
+ assertThat(options.getTransformNameMapping(), hasEntry("bar", "baz"));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java
new file mode 100644
index 0000000..e7db40f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.dataflow.options;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.ResetDateTimeProvider;
+import org.apache.beam.sdk.testing.RestoreSystemProperties;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link DataflowPipelineOptions}. */
+@RunWith(JUnit4.class)
+public class DataflowPipelineOptionsTest {
+ @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
+ @Rule public ResetDateTimeProvider resetDateTimeProviderRule = new ResetDateTimeProvider();
+
+ @Test
+ public void testJobNameIsSet() {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setJobName("TestJobName");
+ assertEquals("TestJobName", options.getJobName());
+ }
+
+ @Test
+ public void testUserNameIsNotSet() {
+ resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
+ System.getProperties().remove("user.name");
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setAppName("TestApplication");
+ assertEquals("testapplication--1208190706", options.getJobName());
+ assertTrue(options.getJobName().length() <= 40);
+ }
+
+ @Test
+ public void testAppNameAndUserNameAreLong() {
+ resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
+ System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde");
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setAppName("1234567890123456789012345678901234567890");
+ assertEquals(
+ "a234567890123456789012345678901234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706",
+ options.getJobName());
+ }
+
+ @Test
+ public void testAppNameIsLong() {
+ resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
+ System.getProperties().put("user.name", "abcde");
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setAppName("1234567890123456789012345678901234567890");
+ assertEquals("a234567890123456789012345678901234567890-abcde-1208190706", options.getJobName());
+ }
+
+ @Test
+ public void testUserNameIsLong() {
+ resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
+ System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde");
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setAppName("1234567890");
+ assertEquals("a234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706", options.getJobName());
+ }
+
+ @Test
+ public void testUtf8UserNameAndApplicationNameIsNormalized() {
+ resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
+ System.getProperties().put("user.name", "ði ıntəˈnæʃənəl ");
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setAppName("fəˈnɛtık əsoʊsiˈeıʃn");
+ assertEquals("f00n0t0k00so0si0e00n-0i00nt00n000n0l0-1208190706", options.getJobName());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java
new file mode 100644
index 0000000..58f8514
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.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.dataflow.options;
+
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.hamcrest.Matchers;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link DataflowProfilingOptions}.
+ */
+@RunWith(JUnit4.class)
+public class DataflowProfilingOptionsTest {
+
+ private static final ObjectMapper MAPPER = new ObjectMapper();
+
+ @Test
+ public void testOptionsObject() throws Exception {
+ DataflowPipelineOptions options = PipelineOptionsFactory.fromArgs(new String[] {
+ "--enableProfilingAgent", "--profilingAgentConfiguration={\"interval\": 21}"})
+ .as(DataflowPipelineOptions.class);
+ assertTrue(options.getEnableProfilingAgent());
+
+ String json = MAPPER.writeValueAsString(options);
+ assertThat(json, Matchers.containsString(
+ "\"profilingAgentConfiguration\":{\"interval\":21}"));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java
new file mode 100644
index 0000000..5b12ad5
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.dataflow.options;
+
+import static org.apache.beam.runners.dataflow.options.DataflowWorkerLoggingOptions.Level.WARN;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.beam.runners.dataflow.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides;
+
+import com.google.common.collect.ImmutableMap;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link DataflowWorkerLoggingOptions}. */
+@RunWith(JUnit4.class)
+public class DataflowWorkerLoggingOptionsTest {
+ private static final ObjectMapper MAPPER = new ObjectMapper();
+ @Rule public ExpectedException expectedException = ExpectedException.none();
+
+ @Test
+ public void testWorkerLogLevelOverrideWithInvalidLogLevel() {
+ expectedException.expect(IllegalArgumentException.class);
+ expectedException.expectMessage("Unsupported log level");
+ WorkerLogLevelOverrides.from(ImmutableMap.of("Name", "FakeLevel"));
+ }
+
+ @Test
+ public void testWorkerLogLevelOverrideForClass() throws Exception {
+ assertEquals("{\"org.junit.Test\":\"WARN\"}",
+ MAPPER.writeValueAsString(
+ new WorkerLogLevelOverrides().addOverrideForClass(Test.class, WARN)));
+ }
+
+ @Test
+ public void testWorkerLogLevelOverrideForPackage() throws Exception {
+ assertEquals("{\"org.junit\":\"WARN\"}",
+ MAPPER.writeValueAsString(
+ new WorkerLogLevelOverrides().addOverrideForPackage(Test.class.getPackage(), WARN)));
+ }
+
+ @Test
+ public void testWorkerLogLevelOverrideForName() throws Exception {
+ assertEquals("{\"A\":\"WARN\"}",
+ MAPPER.writeValueAsString(
+ new WorkerLogLevelOverrides().addOverrideForName("A", WARN)));
+ }
+
+ @Test
+ public void testSerializationAndDeserializationOf() throws Exception {
+ String testValue = "{\"A\":\"WARN\"}";
+ assertEquals(testValue,
+ MAPPER.writeValueAsString(
+ MAPPER.readValue(testValue, WorkerLogLevelOverrides.class)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..d6de501
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowPipelineRunnerTest.java
@@ -0,0 +1,381 @@
+/*
+ * 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.dataflow.testing;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+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.State;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.TimeUtil;
+import org.apache.beam.sdk.util.Transport;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.api.client.http.LowLevelHttpResponse;
+import com.google.api.client.json.Json;
+import com.google.api.client.testing.http.MockHttpTransport;
+import com.google.api.client.testing.http.MockLowLevelHttpRequest;
+import com.google.api.client.testing.http.MockLowLevelHttpResponse;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.JobMetrics;
+import com.google.api.services.dataflow.model.MetricStructuredName;
+import com.google.api.services.dataflow.model.MetricUpdate;
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+
+import org.joda.time.Instant;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+/** Tests for {@link TestDataflowPipelineRunner}. */
+@RunWith(JUnit4.class)
+public class TestDataflowPipelineRunnerTest {
+ @Rule public ExpectedException expectedException = ExpectedException.none();
+ @Mock private MockHttpTransport transport;
+ @Mock private MockLowLevelHttpRequest request;
+ @Mock private GcsUtil mockGcsUtil;
+
+ private TestDataflowPipelineOptions options;
+ private Dataflow service;
+
+ @Before
+ public void setUp() throws Exception {
+ MockitoAnnotations.initMocks(this);
+ when(transport.buildRequest(anyString(), anyString())).thenReturn(request);
+ doCallRealMethod().when(request).getContentAsString();
+ service = new Dataflow(transport, Transport.getJsonFactory(), null);
+
+ options = PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
+ options.setAppName("TestAppName");
+ options.setProject("test-project");
+ options.setTempLocation("gs://test/temp/location");
+ options.setTempRoot("gs://test");
+ options.setGcpCredential(new TestCredential());
+ options.setDataflowClient(service);
+ options.setRunner(TestDataflowPipelineRunner.class);
+ options.setPathValidatorClass(NoopPathValidator.class);
+ }
+
+ @Test
+ public void testToString() {
+ assertEquals("TestDataflowPipelineRunner#TestAppName",
+ new TestDataflowPipelineRunner(options).toString());
+ }
+
+ @Test
+ public void testRunBatchJobThatSucceeds() throws Exception {
+ Pipeline p = TestPipeline.create(options);
+ PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+ PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+ 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();
+ when(request.execute()).thenReturn(
+ generateMockMetricResponse(true /* success */, true /* tentative */));
+ assertEquals(mockJob, runner.run(p, mockRunner));
+ }
+
+ @Test
+ public void testRunBatchJobThatFails() throws Exception {
+ Pipeline p = TestPipeline.create(options);
+ PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+ PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+ 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();
+ try {
+ runner.run(p, mockRunner);
+ } catch (AssertionError expected) {
+ return;
+ }
+ // Note that fail throws an AssertionError which is why it is placed out here
+ // instead of inside the try-catch block.
+ fail("AssertionError expected");
+ }
+
+ @Test
+ public void testBatchPipelineFailsIfException() throws Exception {
+ Pipeline p = TestPipeline.create(options);
+ PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+ PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+ DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+ when(mockJob.getDataflowClient()).thenReturn(service);
+ when(mockJob.getState()).thenReturn(State.RUNNING);
+ when(mockJob.getProjectId()).thenReturn("test-project");
+ when(mockJob.getJobId()).thenReturn("test-job");
+ when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+ .thenAnswer(new Answer<State>() {
+ @Override
+ public State answer(InvocationOnMock invocation) {
+ JobMessage message = new JobMessage();
+ message.setMessageText("FooException");
+ message.setTime(TimeUtil.toCloudTime(Instant.now()));
+ message.setMessageImportance("JOB_MESSAGE_ERROR");
+ ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
+ .process(Arrays.asList(message));
+ return State.CANCELLED;
+ }
+ });
+
+ DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+ when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+ when(request.execute()).thenReturn(
+ generateMockMetricResponse(false /* success */, true /* tentative */));
+ TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+ try {
+ runner.run(p, mockRunner);
+ } catch (AssertionError expected) {
+ assertThat(expected.getMessage(), containsString("FooException"));
+ verify(mockJob, atLeastOnce()).cancel();
+ return;
+ }
+ // Note that fail throws an AssertionError which is why it is placed out here
+ // instead of inside the try-catch block.
+ fail("AssertionError expected");
+ }
+
+ @Test
+ public void testRunStreamingJobThatSucceeds() throws Exception {
+ options.setStreaming(true);
+ Pipeline p = TestPipeline.create(options);
+ PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+ PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+ DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+ when(mockJob.getDataflowClient()).thenReturn(service);
+ when(mockJob.getState()).thenReturn(State.RUNNING);
+ 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);
+
+ when(request.execute()).thenReturn(
+ generateMockMetricResponse(true /* success */, true /* tentative */));
+ TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+ runner.run(p, mockRunner);
+ }
+
+ @Test
+ public void testRunStreamingJobThatFails() throws Exception {
+ options.setStreaming(true);
+ Pipeline p = TestPipeline.create(options);
+ PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+ PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+ DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+ when(mockJob.getDataflowClient()).thenReturn(service);
+ when(mockJob.getState()).thenReturn(State.RUNNING);
+ 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);
+
+ when(request.execute()).thenReturn(
+ generateMockMetricResponse(false /* success */, true /* tentative */));
+ TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+ try {
+ runner.run(p, mockRunner);
+ } catch (AssertionError expected) {
+ return;
+ }
+ // Note that fail throws an AssertionError which is why it is placed out here
+ // instead of inside the try-catch block.
+ fail("AssertionError expected");
+ }
+
+ @Test
+ public void testCheckingForSuccessWhenPAssertSucceeds() throws Exception {
+ DataflowPipelineJob job =
+ spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+ Pipeline p = TestPipeline.create(options);
+ PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+ PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+ TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+ when(request.execute()).thenReturn(
+ generateMockMetricResponse(true /* success */, true /* tentative */));
+ doReturn(State.DONE).when(job).getState();
+ assertEquals(Optional.of(true), runner.checkForSuccess(job));
+ }
+
+ @Test
+ public void testCheckingForSuccessWhenPAssertFails() throws Exception {
+ DataflowPipelineJob job =
+ spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+ Pipeline p = TestPipeline.create(options);
+ PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+ PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+ TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+ when(request.execute()).thenReturn(
+ generateMockMetricResponse(false /* success */, true /* tentative */));
+ doReturn(State.DONE).when(job).getState();
+ assertEquals(Optional.of(false), runner.checkForSuccess(job));
+ }
+
+ @Test
+ public void testCheckingForSuccessSkipsNonTentativeMetrics() throws Exception {
+ DataflowPipelineJob job =
+ spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+ Pipeline p = TestPipeline.create(options);
+ PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+ PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+ TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+ when(request.execute()).thenReturn(
+ generateMockMetricResponse(true /* success */, false /* tentative */));
+ doReturn(State.RUNNING).when(job).getState();
+ assertEquals(Optional.absent(), runner.checkForSuccess(job));
+ }
+
+ private LowLevelHttpResponse generateMockMetricResponse(boolean success, boolean tentative)
+ throws Exception {
+ MetricStructuredName name = new MetricStructuredName();
+ name.setName(success ? "PAssertSuccess" : "PAssertFailure");
+ name.setContext(
+ tentative ? ImmutableMap.of("tentative", "") : ImmutableMap.<String, String>of());
+
+ MetricUpdate metric = new MetricUpdate();
+ metric.setName(name);
+ metric.setScalar(BigDecimal.ONE);
+
+ MockLowLevelHttpResponse response = new MockLowLevelHttpResponse();
+ response.setContentType(Json.MEDIA_TYPE);
+ JobMetrics jobMetrics = new JobMetrics();
+ jobMetrics.setMetrics(Lists.newArrayList(metric));
+ // N.B. Setting the factory is necessary in order to get valid JSON.
+ jobMetrics.setFactory(Transport.getJsonFactory());
+ response.setContent(jobMetrics.toPrettyString());
+ return response;
+ }
+
+ @Test
+ public void testStreamingPipelineFailsIfServiceFails() throws Exception {
+ DataflowPipelineJob job =
+ spy(new DataflowPipelineJob("test-project", "test-job", service, null));
+ Pipeline p = TestPipeline.create(options);
+ PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+ PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+ TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+ when(request.execute()).thenReturn(
+ generateMockMetricResponse(true /* success */, false /* tentative */));
+ doReturn(State.FAILED).when(job).getState();
+ assertEquals(Optional.of(false), runner.checkForSuccess(job));
+ }
+
+ @Test
+ public void testStreamingPipelineFailsIfException() throws Exception {
+ options.setStreaming(true);
+ Pipeline p = TestPipeline.create(options);
+ PCollection<Integer> pc = p.apply(Create.of(1, 2, 3));
+ PAssert.that(pc).containsInAnyOrder(1, 2, 3);
+
+ DataflowPipelineJob mockJob = Mockito.mock(DataflowPipelineJob.class);
+ when(mockJob.getDataflowClient()).thenReturn(service);
+ when(mockJob.getState()).thenReturn(State.RUNNING);
+ when(mockJob.getProjectId()).thenReturn("test-project");
+ when(mockJob.getJobId()).thenReturn("test-job");
+ when(mockJob.waitToFinish(any(Long.class), any(TimeUnit.class), any(JobMessagesHandler.class)))
+ .thenAnswer(new Answer<State>() {
+ @Override
+ public State answer(InvocationOnMock invocation) {
+ JobMessage message = new JobMessage();
+ message.setMessageText("FooException");
+ message.setTime(TimeUtil.toCloudTime(Instant.now()));
+ message.setMessageImportance("JOB_MESSAGE_ERROR");
+ ((MonitoringUtil.JobMessagesHandler) invocation.getArguments()[2])
+ .process(Arrays.asList(message));
+ return State.CANCELLED;
+ }
+ });
+
+ DataflowPipelineRunner mockRunner = Mockito.mock(DataflowPipelineRunner.class);
+ when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob);
+
+ when(request.execute()).thenReturn(
+ generateMockMetricResponse(false /* success */, true /* tentative */));
+ TestDataflowPipelineRunner runner = (TestDataflowPipelineRunner) p.getRunner();
+ try {
+ runner.run(p, mockRunner);
+ } catch (AssertionError expected) {
+ assertThat(expected.getMessage(), containsString("FooException"));
+ verify(mockJob, atLeastOnce()).cancel();
+ return;
+ }
+ // Note that fail throws an AssertionError which is why it is placed out here
+ // instead of inside the try-catch block.
+ fail("AssertionError expected");
+ }
+}
[09/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java
deleted file mode 100644
index d647b0d..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/testing/TestDataflowPipelineRunner.java
+++ /dev/null
@@ -1,271 +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.testing;
-
-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.DataflowJobExecutionException;
-import org.apache.beam.sdk.runners.DataflowPipelineJob;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
-import org.apache.beam.sdk.runners.PipelineRunner;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.MonitoringUtil;
-import org.apache.beam.sdk.util.MonitoringUtil.JobMessagesHandler;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.JobMetrics;
-import com.google.api.services.dataflow.model.MetricUpdate;
-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;
-
-/**
- * {@link TestDataflowPipelineRunner} is a pipeline runner that wraps a
- * {@link DataflowPipelineRunner} when running tests against the {@link TestPipeline}.
- *
- * @see TestPipeline
- */
-public class TestDataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob> {
- private static final String TENTATIVE_COUNTER = "tentative";
- private static final Logger LOG = LoggerFactory.getLogger(TestDataflowPipelineRunner.class);
- private static final Map<String, PipelineResult> EXECUTION_RESULTS =
- new ConcurrentHashMap<String, PipelineResult>();
-
- private final TestDataflowPipelineOptions options;
- private final DataflowPipelineRunner runner;
- private int expectedNumberOfAssertions = 0;
-
- TestDataflowPipelineRunner(TestDataflowPipelineOptions options) {
- this.options = options;
- this.runner = DataflowPipelineRunner.fromOptions(options);
- }
-
- /**
- * Constructs a runner from the provided options.
- */
- public static TestDataflowPipelineRunner fromOptions(
- PipelineOptions options) {
- TestDataflowPipelineOptions dataflowOptions = options.as(TestDataflowPipelineOptions.class);
- dataflowOptions.setStagingLocation(Joiner.on("/").join(
- new String[]{dataflowOptions.getTempRoot(),
- dataflowOptions.getJobName(), "output", "results"}));
-
- 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);
- }
-
- DataflowPipelineJob run(Pipeline pipeline, DataflowPipelineRunner runner) {
-
- final DataflowPipelineJob job;
- try {
- job = runner.run(pipeline);
- } catch (DataflowJobExecutionException ex) {
- throw new IllegalStateException("The dataflow failed.");
- }
-
- LOG.info("Running Dataflow job {} with {} expected assertions.",
- job.getJobId(), expectedNumberOfAssertions);
-
- CancelWorkflowOnError messageHandler = new CancelWorkflowOnError(
- job, new MonitoringUtil.PrintHandler(options.getJobMessageOutput()));
-
- try {
- final Optional<Boolean> result;
-
- if (options.isStreaming()) {
- Future<Optional<Boolean>> resultFuture = options.getExecutorService().submit(
- new Callable<Optional<Boolean>>() {
- @Override
- public Optional<Boolean> call() throws Exception {
- try {
- for (;;) {
- Optional<Boolean> result = checkForSuccess(job);
- if (result.isPresent()) {
- return result;
- }
- Thread.sleep(10000L);
- }
- } finally {
- LOG.info("Cancelling Dataflow job {}", job.getJobId());
- job.cancel();
- }
- }
- });
- State finalState = job.waitToFinish(10L, TimeUnit.MINUTES, messageHandler);
- if (finalState == null || finalState == State.RUNNING) {
- LOG.info("Dataflow job {} took longer than 10 minutes to complete, cancelling.",
- job.getJobId());
- job.cancel();
- }
- result = resultFuture.get();
- } else {
- job.waitToFinish(-1, TimeUnit.SECONDS, messageHandler);
- result = checkForSuccess(job);
- }
- if (!result.isPresent()) {
- throw new IllegalStateException(
- "The dataflow did not output a success or failure metric.");
- } else if (!result.get()) {
- throw new AssertionError(messageHandler.getErrorMessage() == null ?
- "The dataflow did not return a failure reason."
- : messageHandler.getErrorMessage());
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new RuntimeException(e);
- } catch (ExecutionException e) {
- Throwables.propagateIfPossible(e.getCause());
- throw new RuntimeException(e.getCause());
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- EXECUTION_RESULTS.put(options.getJobName(), job);
- return job;
- }
-
- @Override
- public <OutputT extends POutput, InputT extends PInput> OutputT apply(
- PTransform<InputT, OutputT> transform, InputT input) {
- if (transform instanceof PAssert.OneSideInputAssert
- || transform instanceof PAssert.TwoSideInputAssert) {
- expectedNumberOfAssertions += 1;
- }
-
- return runner.apply(transform, input);
- }
-
- Optional<Boolean> checkForSuccess(DataflowPipelineJob job)
- throws IOException {
- State state = job.getState();
- if (state == State.FAILED || state == State.CANCELLED) {
- LOG.info("The pipeline failed");
- return Optional.of(false);
- }
-
- JobMetrics metrics = job.getDataflowClient().projects().jobs()
- .getMetrics(job.getProjectId(), job.getJobId()).execute();
-
- if (metrics == null || metrics.getMetrics() == null) {
- LOG.warn("Metrics not present for Dataflow job {}.", job.getJobId());
- } else {
- int successes = 0;
- int failures = 0;
- for (MetricUpdate metric : metrics.getMetrics()) {
- if (metric.getName() == null || metric.getName().getContext() == null
- || !metric.getName().getContext().containsKey(TENTATIVE_COUNTER)) {
- // Don't double count using the non-tentative version of the metric.
- continue;
- }
- if (PAssert.SUCCESS_COUNTER.equals(metric.getName().getName())) {
- successes += ((BigDecimal) metric.getScalar()).intValue();
- } else if (PAssert.FAILURE_COUNTER.equals(metric.getName().getName())) {
- failures += ((BigDecimal) metric.getScalar()).intValue();
- }
- }
-
- if (failures > 0) {
- LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
- + "{} expected assertions.", job.getJobId(), successes, failures,
- expectedNumberOfAssertions);
- return Optional.of(false);
- } else if (successes >= expectedNumberOfAssertions) {
- LOG.info("Found result while running Dataflow job {}. Found {} success, {} failures out of "
- + "{} expected assertions.", job.getJobId(), successes, failures,
- expectedNumberOfAssertions);
- return Optional.of(true);
- }
-
- LOG.info("Running Dataflow job {}. Found {} success, {} failures out of {} expected "
- + "assertions.", job.getJobId(), successes, failures, expectedNumberOfAssertions);
- }
-
- return Optional.<Boolean>absent();
- }
-
- @Override
- public String toString() {
- return "TestDataflowPipelineRunner#" + options.getAppName();
- }
-
- /**
- * Cancels the workflow on the first error message it sees.
- *
- * <p>Creates an error message representing the concatenation of all error messages seen.
- */
- private static class CancelWorkflowOnError implements JobMessagesHandler {
- private final DataflowPipelineJob job;
- private final JobMessagesHandler messageHandler;
- private final StringBuffer errorMessage;
- private CancelWorkflowOnError(DataflowPipelineJob job, JobMessagesHandler messageHandler) {
- this.job = job;
- this.messageHandler = messageHandler;
- this.errorMessage = new StringBuffer();
- }
-
- @Override
- public void process(List<JobMessage> messages) {
- messageHandler.process(messages);
- for (JobMessage message : messages) {
- if (message.getMessageImportance() != null
- && message.getMessageImportance().equals("JOB_MESSAGE_ERROR")) {
- LOG.info("Dataflow job {} threw exception. Failure message was: {}",
- job.getJobId(), message.getMessageText());
- errorMessage.append(message.getMessageText());
- }
- }
- if (errorMessage.length() > 0) {
- LOG.info("Cancelling Dataflow job {}", job.getJobId());
- try {
- job.cancel();
- } catch (Exception ignore) {
- // The TestDataflowPipelineRunner will thrown an AssertionError with the job failure
- // messages.
- }
- }
- }
-
- private String getErrorMessage() {
- return errorMessage.toString();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java
deleted file mode 100644
index d0388a3..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowPathValidator.java
+++ /dev/null
@@ -1,99 +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.util;
-
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-
-import com.google.common.base.Preconditions;
-
-import java.io.IOException;
-
-/**
- * GCP implementation of {@link PathValidator}. Only GCS paths are allowed.
- */
-public class DataflowPathValidator implements PathValidator {
-
- private DataflowPipelineOptions dataflowOptions;
-
- DataflowPathValidator(DataflowPipelineOptions options) {
- this.dataflowOptions = options;
- }
-
- public static DataflowPathValidator fromOptions(PipelineOptions options) {
- return new DataflowPathValidator(options.as(DataflowPipelineOptions.class));
- }
-
- /**
- * Validates the the input GCS path is accessible and that the path
- * is well formed.
- */
- @Override
- public String validateInputFilePatternSupported(String filepattern) {
- GcsPath gcsPath = getGcsPath(filepattern);
- Preconditions.checkArgument(
- dataflowOptions.getGcsUtil().isGcsPatternSupported(gcsPath.getObject()));
- String returnValue = verifyPath(filepattern);
- verifyPathIsAccessible(filepattern, "Could not find file %s");
- return returnValue;
- }
-
- /**
- * Validates the the output GCS path is accessible and that the path
- * is well formed.
- */
- @Override
- public String validateOutputFilePrefixSupported(String filePrefix) {
- String returnValue = verifyPath(filePrefix);
- verifyPathIsAccessible(filePrefix, "Output path does not exist or is not writeable: %s");
- return returnValue;
- }
-
- @Override
- public String verifyPath(String path) {
- GcsPath gcsPath = getGcsPath(path);
- Preconditions.checkArgument(gcsPath.isAbsolute(),
- "Must provide absolute paths for Dataflow");
- Preconditions.checkArgument(!gcsPath.getObject().contains("//"),
- "Dataflow Service does not allow objects with consecutive slashes");
- return gcsPath.toResourceName();
- }
-
- private void verifyPathIsAccessible(String path, String errorMessage) {
- GcsPath gcsPath = getGcsPath(path);
- try {
- Preconditions.checkArgument(dataflowOptions.getGcsUtil().bucketExists(gcsPath),
- errorMessage, path);
- } catch (IOException e) {
- throw new RuntimeException(
- String.format("Unable to verify that GCS bucket gs://%s exists.", gcsPath.getBucket()),
- e);
- }
- }
-
- private GcsPath getGcsPath(String path) {
- try {
- return GcsPath.fromUri(path);
- } catch (IllegalArgumentException e) {
- throw new IllegalArgumentException(String.format(
- "%s expected a valid 'gs://' path but was given '%s'",
- dataflowOptions.getRunner().getSimpleName(), path), e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java
deleted file mode 100644
index 8fcfccf..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/DataflowTransport.java
+++ /dev/null
@@ -1,113 +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.util;
-
-import static org.apache.beam.sdk.util.Transport.getJsonFactory;
-import static org.apache.beam.sdk.util.Transport.getTransport;
-
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-
-import com.google.api.client.auth.oauth2.Credential;
-import com.google.api.client.http.HttpRequestInitializer;
-import com.google.api.services.clouddebugger.v2.Clouddebugger;
-import com.google.api.services.dataflow.Dataflow;
-import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer;
-import com.google.common.collect.ImmutableList;
-
-import java.net.MalformedURLException;
-import java.net.URL;
-
-/**
- * Helpers for cloud communication.
- */
-public class DataflowTransport {
-
-
- private static class ApiComponents {
- public String rootUrl;
- public String servicePath;
-
- public ApiComponents(String root, String path) {
- this.rootUrl = root;
- this.servicePath = path;
- }
- }
-
- private static ApiComponents apiComponentsFromUrl(String urlString) {
- try {
- URL url = new URL(urlString);
- String rootUrl = url.getProtocol() + "://" + url.getHost() +
- (url.getPort() > 0 ? ":" + url.getPort() : "");
- return new ApiComponents(rootUrl, url.getPath());
- } catch (MalformedURLException e) {
- throw new RuntimeException("Invalid URL: " + urlString);
- }
- }
-
- /**
- * Returns a Google Cloud Dataflow client builder.
- */
- public static Dataflow.Builder newDataflowClient(DataflowPipelineOptions options) {
- String servicePath = options.getDataflowEndpoint();
- ApiComponents components;
- if (servicePath.contains("://")) {
- components = apiComponentsFromUrl(servicePath);
- } else {
- components = new ApiComponents(options.getApiRootUrl(), servicePath);
- }
-
- return new Dataflow.Builder(getTransport(),
- getJsonFactory(),
- chainHttpRequestInitializer(
- options.getGcpCredential(),
- // Do not log 404. It clutters the output and is possibly even required by the caller.
- new RetryHttpRequestInitializer(ImmutableList.of(404))))
- .setApplicationName(options.getAppName())
- .setRootUrl(components.rootUrl)
- .setServicePath(components.servicePath)
- .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
- }
-
- public static Clouddebugger.Builder newClouddebuggerClient(DataflowPipelineOptions options) {
- return new Clouddebugger.Builder(getTransport(),
- getJsonFactory(),
- chainHttpRequestInitializer(options.getGcpCredential(), new RetryHttpRequestInitializer()))
- .setApplicationName(options.getAppName())
- .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
- }
-
- /**
- * Returns a Dataflow client that does not automatically retry failed
- * requests.
- */
- public static Dataflow.Builder
- newRawDataflowClient(DataflowPipelineOptions options) {
- return newDataflowClient(options)
- .setHttpRequestInitializer(options.getGcpCredential())
- .setGoogleClientRequestInitializer(options.getGoogleApiTrace());
- }
-
- private static HttpRequestInitializer chainHttpRequestInitializer(
- Credential credential, HttpRequestInitializer httpRequestInitializer) {
- if (credential == null) {
- return httpRequestInitializer;
- } else {
- return new ChainingHttpRequestInitializer(credential, httpRequestInitializer);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java
deleted file mode 100644
index 4f1f673..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/GcsStager.java
+++ /dev/null
@@ -1,55 +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.util;
-
-import org.apache.beam.sdk.options.DataflowPipelineDebugOptions;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.common.base.Preconditions;
-
-import java.util.List;
-
-/**
- * Utility class for staging files to GCS.
- */
-public class GcsStager implements Stager {
- private DataflowPipelineOptions options;
-
- private GcsStager(DataflowPipelineOptions options) {
- this.options = options;
- }
-
- public static GcsStager fromOptions(PipelineOptions options) {
- return new GcsStager(options.as(DataflowPipelineOptions.class));
- }
-
- @Override
- public List<DataflowPackage> stageFiles() {
- Preconditions.checkNotNull(options.getStagingLocation());
- List<String> filesToStage = options.getFilesToStage();
- String windmillBinary =
- options.as(DataflowPipelineDebugOptions.class).getOverrideWindmillBinary();
- if (windmillBinary != null) {
- filesToStage.add("windmill_main=" + windmillBinary);
- }
- return PackageUtil.stageClasspathElements(
- options.getFilesToStage(), options.getStagingLocation());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java
deleted file mode 100644
index 5afca52..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/MonitoringUtil.java
+++ /dev/null
@@ -1,236 +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.util;
-
-import static org.apache.beam.sdk.util.TimeUtil.fromCloudTime;
-
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-
-import com.google.api.services.dataflow.Dataflow;
-import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages;
-import com.google.api.services.dataflow.model.JobMessage;
-import com.google.api.services.dataflow.model.ListJobMessagesResponse;
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableMap;
-
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.io.PrintStream;
-import java.io.UnsupportedEncodingException;
-import java.net.URLEncoder;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-
-import javax.annotation.Nullable;
-
-/**
- * A helper class for monitoring jobs submitted to the service.
- */
-public final class MonitoringUtil {
-
- private static final String GCLOUD_DATAFLOW_PREFIX = "gcloud alpha dataflow";
- private static final String ENDPOINT_OVERRIDE_ENV_VAR =
- "CLOUDSDK_API_ENDPOINT_OVERRIDES_DATAFLOW";
-
- private static final Map<String, State> DATAFLOW_STATE_TO_JOB_STATE =
- ImmutableMap
- .<String, State>builder()
- .put("JOB_STATE_UNKNOWN", State.UNKNOWN)
- .put("JOB_STATE_STOPPED", State.STOPPED)
- .put("JOB_STATE_RUNNING", State.RUNNING)
- .put("JOB_STATE_DONE", State.DONE)
- .put("JOB_STATE_FAILED", State.FAILED)
- .put("JOB_STATE_CANCELLED", State.CANCELLED)
- .put("JOB_STATE_UPDATED", State.UPDATED)
- .build();
-
- private String projectId;
- private Messages messagesClient;
-
- /**
- * An interface that can be used for defining callbacks to receive a list
- * of JobMessages containing monitoring information.
- */
- public interface JobMessagesHandler {
- /** Process the rows. */
- void process(List<JobMessage> messages);
- }
-
- /** A handler that prints monitoring messages to a stream. */
- public static class PrintHandler implements JobMessagesHandler {
- private PrintStream out;
-
- /**
- * Construct the handler.
- *
- * @param stream The stream to write the messages to.
- */
- public PrintHandler(PrintStream stream) {
- out = stream;
- }
-
- @Override
- public void process(List<JobMessage> messages) {
- for (JobMessage message : messages) {
- if (message.getMessageText() == null || message.getMessageText().isEmpty()) {
- continue;
- }
- String importanceString = null;
- if (message.getMessageImportance() == null) {
- continue;
- } else if (message.getMessageImportance().equals("JOB_MESSAGE_ERROR")) {
- importanceString = "Error: ";
- } else if (message.getMessageImportance().equals("JOB_MESSAGE_WARNING")) {
- importanceString = "Warning: ";
- } else if (message.getMessageImportance().equals("JOB_MESSAGE_BASIC")) {
- importanceString = "Basic: ";
- } else if (message.getMessageImportance().equals("JOB_MESSAGE_DETAILED")) {
- importanceString = "Detail: ";
- } else {
- // TODO: Remove filtering here once getJobMessages supports minimum
- // importance.
- continue;
- }
- @Nullable Instant time = TimeUtil.fromCloudTime(message.getTime());
- if (time == null) {
- out.print("UNKNOWN TIMESTAMP: ");
- } else {
- out.print(time + ": ");
- }
- if (importanceString != null) {
- out.print(importanceString);
- }
- out.println(message.getMessageText());
- }
- out.flush();
- }
- }
-
- /** Construct a helper for monitoring. */
- public MonitoringUtil(String projectId, Dataflow dataflow) {
- this(projectId, dataflow.projects().jobs().messages());
- }
-
- // @VisibleForTesting
- MonitoringUtil(String projectId, Messages messagesClient) {
- this.projectId = projectId;
- this.messagesClient = messagesClient;
- }
-
- /**
- * Comparator for sorting rows in increasing order based on timestamp.
- */
- public static class TimeStampComparator implements Comparator<JobMessage> {
- @Override
- public int compare(JobMessage o1, JobMessage o2) {
- @Nullable Instant t1 = fromCloudTime(o1.getTime());
- if (t1 == null) {
- return -1;
- }
- @Nullable Instant t2 = fromCloudTime(o2.getTime());
- if (t2 == null) {
- return 1;
- }
- return t1.compareTo(t2);
- }
- }
-
- /**
- * Return job messages sorted in ascending order by timestamp.
- * @param jobId The id of the job to get the messages for.
- * @param startTimestampMs Return only those messages with a
- * timestamp greater than this value.
- * @return collection of messages
- * @throws IOException
- */
- public ArrayList<JobMessage> getJobMessages(
- String jobId, long startTimestampMs) throws IOException {
- // TODO: Allow filtering messages by importance
- Instant startTimestamp = new Instant(startTimestampMs);
- ArrayList<JobMessage> allMessages = new ArrayList<>();
- String pageToken = null;
- while (true) {
- Messages.List listRequest = messagesClient.list(projectId, jobId);
- if (pageToken != null) {
- listRequest.setPageToken(pageToken);
- }
- ListJobMessagesResponse response = listRequest.execute();
-
- if (response == null || response.getJobMessages() == null) {
- return allMessages;
- }
-
- for (JobMessage m : response.getJobMessages()) {
- @Nullable Instant timestamp = fromCloudTime(m.getTime());
- if (timestamp == null) {
- continue;
- }
- if (timestamp.isAfter(startTimestamp)) {
- allMessages.add(m);
- }
- }
-
- if (response.getNextPageToken() == null) {
- break;
- } else {
- pageToken = response.getNextPageToken();
- }
- }
-
- Collections.sort(allMessages, new TimeStampComparator());
- return allMessages;
- }
-
- public static String getJobMonitoringPageURL(String projectName, String jobId) {
- try {
- // Project name is allowed in place of the project id: the user will be redirected to a URL
- // that has the project name replaced with project id.
- return String.format(
- "https://console.developers.google.com/project/%s/dataflow/job/%s",
- URLEncoder.encode(projectName, "UTF-8"),
- URLEncoder.encode(jobId, "UTF-8"));
- } catch (UnsupportedEncodingException e) {
- // Should never happen.
- throw new AssertionError("UTF-8 encoding is not supported by the environment", e);
- }
- }
-
- public static String getGcloudCancelCommand(DataflowPipelineOptions options, String jobId) {
-
- // If using a different Dataflow API than default, prefix command with an API override.
- String dataflowApiOverridePrefix = "";
- String apiUrl = options.getDataflowClient().getBaseUrl();
- if (!apiUrl.equals(Dataflow.DEFAULT_BASE_URL)) {
- dataflowApiOverridePrefix = String.format("%s=%s ", ENDPOINT_OVERRIDE_ENV_VAR, apiUrl);
- }
-
- // Assemble cancel command from optional prefix and project/job parameters.
- return String.format("%s%s jobs --project=%s cancel %s",
- dataflowApiOverridePrefix, GCLOUD_DATAFLOW_PREFIX, options.getProject(), jobId);
- }
-
- public static State toState(String stateName) {
- return MoreObjects.firstNonNull(DATAFLOW_STATE_TO_JOB_STATE.get(stateName),
- State.UNKNOWN);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java
deleted file mode 100644
index 7d492b4..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/PackageUtil.java
+++ /dev/null
@@ -1,328 +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.util;
-
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.Sleeper;
-import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.cloud.hadoop.util.ApiErrorExtractor;
-import com.google.common.hash.Funnels;
-import com.google.common.hash.Hasher;
-import com.google.common.hash.Hashing;
-import com.google.common.io.CountingOutputStream;
-import com.google.common.io.Files;
-
-import com.fasterxml.jackson.core.Base64Variants;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Objects;
-
-/** Helper routines for packages. */
-public class PackageUtil {
- private static final Logger LOG = LoggerFactory.getLogger(PackageUtil.class);
- /**
- * A reasonable upper bound on the number of jars required to launch a Dataflow job.
- */
- public static final int SANE_CLASSPATH_SIZE = 1000;
- /**
- * The initial interval to use between package staging attempts.
- */
- private static final long INITIAL_BACKOFF_INTERVAL_MS = 5000L;
- /**
- * The maximum number of attempts when staging a file.
- */
- private static final int MAX_ATTEMPTS = 5;
-
- /**
- * Translates exceptions from API calls.
- */
- private static final ApiErrorExtractor ERROR_EXTRACTOR = new ApiErrorExtractor();
-
- /**
- * Creates a DataflowPackage containing information about how a classpath element should be
- * staged, including the staging destination as well as its size and hash.
- *
- * @param classpathElement The local path for the classpath element.
- * @param stagingPath The base location for staged classpath elements.
- * @param overridePackageName If non-null, use the given value as the package name
- * instead of generating one automatically.
- * @return The package.
- */
- @Deprecated
- public static DataflowPackage createPackage(File classpathElement,
- String stagingPath, String overridePackageName) {
- return createPackageAttributes(classpathElement, stagingPath, overridePackageName)
- .getDataflowPackage();
- }
-
- /**
- * Compute and cache the attributes of a classpath element that we will need to stage it.
- *
- * @param classpathElement the file or directory to be staged.
- * @param stagingPath The base location for staged classpath elements.
- * @param overridePackageName If non-null, use the given value as the package name
- * instead of generating one automatically.
- * @return a {@link PackageAttributes} that containing metadata about the object to be staged.
- */
- static PackageAttributes createPackageAttributes(File classpathElement,
- String stagingPath, String overridePackageName) {
- try {
- boolean directory = classpathElement.isDirectory();
-
- // Compute size and hash in one pass over file or directory.
- Hasher hasher = Hashing.md5().newHasher();
- OutputStream hashStream = Funnels.asOutputStream(hasher);
- CountingOutputStream countingOutputStream = new CountingOutputStream(hashStream);
-
- if (!directory) {
- // Files are staged as-is.
- Files.asByteSource(classpathElement).copyTo(countingOutputStream);
- } else {
- // Directories are recursively zipped.
- ZipFiles.zipDirectory(classpathElement, countingOutputStream);
- }
-
- long size = countingOutputStream.getCount();
- String hash = Base64Variants.MODIFIED_FOR_URL.encode(hasher.hash().asBytes());
-
- // Create the DataflowPackage with staging name and location.
- String uniqueName = getUniqueContentName(classpathElement, hash);
- String resourcePath = IOChannelUtils.resolve(stagingPath, uniqueName);
- DataflowPackage target = new DataflowPackage();
- target.setName(overridePackageName != null ? overridePackageName : uniqueName);
- target.setLocation(resourcePath);
-
- return new PackageAttributes(size, hash, directory, target);
- } catch (IOException e) {
- throw new RuntimeException("Package setup failure for " + classpathElement, e);
- }
- }
-
- /**
- * Transfers the classpath elements to the staging location.
- *
- * @param classpathElements The elements to stage.
- * @param stagingPath The base location to stage the elements to.
- * @return A list of cloud workflow packages, each representing a classpath element.
- */
- public static List<DataflowPackage> stageClasspathElements(
- Collection<String> classpathElements, String stagingPath) {
- return stageClasspathElements(classpathElements, stagingPath, Sleeper.DEFAULT);
- }
-
- // Visible for testing.
- static List<DataflowPackage> stageClasspathElements(
- Collection<String> classpathElements, String stagingPath,
- Sleeper retrySleeper) {
- LOG.info("Uploading {} files from PipelineOptions.filesToStage to staging location to "
- + "prepare for execution.", classpathElements.size());
-
- if (classpathElements.size() > SANE_CLASSPATH_SIZE) {
- LOG.warn("Your classpath contains {} elements, which Google Cloud Dataflow automatically "
- + "copies to all workers. Having this many entries on your classpath may be indicative "
- + "of an issue in your pipeline. You may want to consider trimming the classpath to "
- + "necessary dependencies only, using --filesToStage pipeline option to override "
- + "what files are being staged, or bundling several dependencies into one.",
- classpathElements.size());
- }
-
- ArrayList<DataflowPackage> packages = new ArrayList<>();
-
- if (stagingPath == null) {
- throw new IllegalArgumentException(
- "Can't stage classpath elements on because no staging location has been provided");
- }
-
- int numUploaded = 0;
- int numCached = 0;
- for (String classpathElement : classpathElements) {
- String packageName = null;
- if (classpathElement.contains("=")) {
- String[] components = classpathElement.split("=", 2);
- packageName = components[0];
- classpathElement = components[1];
- }
-
- File file = new File(classpathElement);
- if (!file.exists()) {
- LOG.warn("Skipping non-existent classpath element {} that was specified.",
- classpathElement);
- continue;
- }
-
- PackageAttributes attributes = createPackageAttributes(file, stagingPath, packageName);
-
- DataflowPackage workflowPackage = attributes.getDataflowPackage();
- packages.add(workflowPackage);
- String target = workflowPackage.getLocation();
-
- // TODO: Should we attempt to detect the Mime type rather than
- // always using MimeTypes.BINARY?
- try {
- try {
- long remoteLength = IOChannelUtils.getSizeBytes(target);
- if (remoteLength == attributes.getSize()) {
- LOG.debug("Skipping classpath element already staged: {} at {}",
- classpathElement, target);
- numCached++;
- continue;
- }
- } catch (FileNotFoundException expected) {
- // If the file doesn't exist, it means we need to upload it.
- }
-
- // Upload file, retrying on failure.
- AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff(
- MAX_ATTEMPTS,
- INITIAL_BACKOFF_INTERVAL_MS);
- while (true) {
- try {
- LOG.debug("Uploading classpath element {} to {}", classpathElement, target);
- try (WritableByteChannel writer = IOChannelUtils.create(target, MimeTypes.BINARY)) {
- copyContent(classpathElement, writer);
- }
- numUploaded++;
- break;
- } catch (IOException e) {
- if (ERROR_EXTRACTOR.accessDenied(e)) {
- String errorMessage = String.format(
- "Uploaded failed due to permissions error, will NOT retry staging "
- + "of classpath %s. Please verify credentials are valid and that you have "
- + "write access to %s. Stale credentials can be resolved by executing "
- + "'gcloud auth login'.", classpathElement, target);
- LOG.error(errorMessage);
- throw new IOException(errorMessage, e);
- } else if (!backoff.atMaxAttempts()) {
- LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}",
- classpathElement, e);
- BackOffUtils.next(retrySleeper, backoff);
- } else {
- // Rethrow last error, to be included as a cause in the catch below.
- LOG.error("Upload failed, will NOT retry staging of classpath: {}",
- classpathElement, e);
- throw e;
- }
- }
- }
- } catch (Exception e) {
- throw new RuntimeException("Could not stage classpath element: " + classpathElement, e);
- }
- }
-
- LOG.info("Uploading PipelineOptions.filesToStage complete: {} files newly uploaded, "
- + "{} files cached",
- numUploaded, numCached);
-
- return packages;
- }
-
- /**
- * Returns a unique name for a file with a given content hash.
- *
- * <p>Directory paths are removed. Example:
- * <pre>
- * dir="a/b/c/d", contentHash="f000" => d-f000.jar
- * file="a/b/c/d.txt", contentHash="f000" => d-f000.txt
- * file="a/b/c/d", contentHash="f000" => d-f000
- * </pre>
- */
- static String getUniqueContentName(File classpathElement, String contentHash) {
- String fileName = Files.getNameWithoutExtension(classpathElement.getAbsolutePath());
- String fileExtension = Files.getFileExtension(classpathElement.getAbsolutePath());
- if (classpathElement.isDirectory()) {
- return fileName + "-" + contentHash + ".jar";
- } else if (fileExtension.isEmpty()) {
- return fileName + "-" + contentHash;
- }
- return fileName + "-" + contentHash + "." + fileExtension;
- }
-
- /**
- * Copies the contents of the classpathElement to the output channel.
- *
- * <p>If the classpathElement is a directory, a Zip stream is constructed on the fly,
- * otherwise the file contents are copied as-is.
- *
- * <p>The output channel is not closed.
- */
- private static void copyContent(String classpathElement, WritableByteChannel outputChannel)
- throws IOException {
- final File classpathElementFile = new File(classpathElement);
- if (classpathElementFile.isDirectory()) {
- ZipFiles.zipDirectory(classpathElementFile, Channels.newOutputStream(outputChannel));
- } else {
- Files.asByteSource(classpathElementFile).copyTo(Channels.newOutputStream(outputChannel));
- }
- }
- /**
- * Holds the metadata necessary to stage a file or confirm that a staged file has not changed.
- */
- static class PackageAttributes {
- private final boolean directory;
- private final long size;
- private final String hash;
- private DataflowPackage dataflowPackage;
-
- public PackageAttributes(long size, String hash, boolean directory,
- DataflowPackage dataflowPackage) {
- this.size = size;
- this.hash = Objects.requireNonNull(hash, "hash");
- this.directory = directory;
- this.dataflowPackage = Objects.requireNonNull(dataflowPackage, "dataflowPackage");
- }
-
- /**
- * @return the dataflowPackage
- */
- public DataflowPackage getDataflowPackage() {
- return dataflowPackage;
- }
-
- /**
- * @return the directory
- */
- public boolean isDirectory() {
- return directory;
- }
-
- /**
- * @return the size
- */
- public long getSize() {
- return size;
- }
-
- /**
- * @return the hash
- */
- public String getHash() {
- return hash;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java
deleted file mode 100644
index 9547ac1..0000000
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/sdk/util/Stager.java
+++ /dev/null
@@ -1,30 +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.util;
-
-import com.google.api.services.dataflow.model.DataflowPackage;
-
-import java.util.List;
-
-/**
- * Interface for staging files needed for running a Dataflow pipeline.
- */
-public interface Stager {
- /* Stage files and return a list of packages. */
- public List<DataflowPackage> stageFiles();
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunnerTest.java
new file mode 100644
index 0000000..bc570e1
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowPipelineRunnerTest.java
@@ -0,0 +1,302 @@
+/*
+ * 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.dataflow;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.testing.TestDataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.TestCredential;
+
+import org.hamcrest.Description;
+import org.hamcrest.Factory;
+import org.hamcrest.Matcher;
+import org.hamcrest.TypeSafeMatcher;
+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.concurrent.TimeUnit;
+
+/**
+ * Tests for BlockingDataflowPipelineRunner.
+ */
+@RunWith(JUnit4.class)
+public class BlockingDataflowPipelineRunnerTest {
+
+ @Rule
+ public ExpectedLogs expectedLogs = ExpectedLogs.none(BlockingDataflowPipelineRunner.class);
+
+ @Rule
+ public ExpectedException expectedThrown = ExpectedException.none();
+
+ /**
+ * A {@link Matcher} for a {@link DataflowJobException} that applies an underlying {@link Matcher}
+ * to the {@link DataflowPipelineJob} returned by {@link DataflowJobException#getJob()}.
+ */
+ private static class DataflowJobExceptionMatcher<T extends DataflowJobException>
+ extends TypeSafeMatcher<T> {
+
+ private final Matcher<DataflowPipelineJob> matcher;
+
+ public DataflowJobExceptionMatcher(Matcher<DataflowPipelineJob> matcher) {
+ this.matcher = matcher;
+ }
+
+ @Override
+ public boolean matchesSafely(T ex) {
+ return matcher.matches(ex.getJob());
+ }
+
+ @Override
+ protected void describeMismatchSafely(T item, Description description) {
+ description.appendText("job ");
+ matcher.describeMismatch(item.getMessage(), description);
+ }
+
+ @Override
+ public void describeTo(Description description) {
+ description.appendText("exception with job matching ");
+ description.appendDescriptionOf(matcher);
+ }
+
+ @Factory
+ public static <T extends DataflowJobException> Matcher<T> expectJob(
+ Matcher<DataflowPipelineJob> matcher) {
+ return new DataflowJobExceptionMatcher<T>(matcher);
+ }
+ }
+
+ /**
+ * A {@link Matcher} for a {@link DataflowPipelineJob} that applies an underlying {@link Matcher}
+ * to the return value of {@link DataflowPipelineJob#getJobId()}.
+ */
+ private static class JobIdMatcher<T extends DataflowPipelineJob> extends TypeSafeMatcher<T> {
+
+ private final Matcher<String> matcher;
+
+ public JobIdMatcher(Matcher<String> matcher) {
+ this.matcher = matcher;
+ }
+
+ @Override
+ public boolean matchesSafely(T job) {
+ return matcher.matches(job.getJobId());
+ }
+
+ @Override
+ protected void describeMismatchSafely(T item, Description description) {
+ description.appendText("jobId ");
+ matcher.describeMismatch(item.getJobId(), description);
+ }
+
+ @Override
+ public void describeTo(Description description) {
+ description.appendText("job with jobId ");
+ description.appendDescriptionOf(matcher);
+ }
+
+ @Factory
+ public static <T extends DataflowPipelineJob> Matcher<T> expectJobId(final String jobId) {
+ return new JobIdMatcher<T>(equalTo(jobId));
+ }
+ }
+
+ /**
+ * A {@link Matcher} for a {@link DataflowJobUpdatedException} that applies an underlying
+ * {@link Matcher} to the {@link DataflowPipelineJob} returned by
+ * {@link DataflowJobUpdatedException#getReplacedByJob()}.
+ */
+ private static class ReplacedByJobMatcher<T extends DataflowJobUpdatedException>
+ extends TypeSafeMatcher<T> {
+
+ private final Matcher<DataflowPipelineJob> matcher;
+
+ public ReplacedByJobMatcher(Matcher<DataflowPipelineJob> matcher) {
+ this.matcher = matcher;
+ }
+
+ @Override
+ public boolean matchesSafely(T ex) {
+ return matcher.matches(ex.getReplacedByJob());
+ }
+
+ @Override
+ protected void describeMismatchSafely(T item, Description description) {
+ description.appendText("job ");
+ matcher.describeMismatch(item.getMessage(), description);
+ }
+
+ @Override
+ public void describeTo(Description description) {
+ description.appendText("exception with replacedByJob() ");
+ description.appendDescriptionOf(matcher);
+ }
+
+ @Factory
+ public static <T extends DataflowJobUpdatedException> Matcher<T> expectReplacedBy(
+ Matcher<DataflowPipelineJob> matcher) {
+ return new ReplacedByJobMatcher<T>(matcher);
+ }
+ }
+
+ /**
+ * Creates a mocked {@link DataflowPipelineJob} with the given {@code projectId} and {@code jobId}
+ * that will immediately terminate in the provided {@code terminalState}.
+ *
+ * <p>The return value may be further mocked.
+ */
+ private DataflowPipelineJob createMockJob(
+ String projectId, String jobId, State terminalState) throws Exception {
+ DataflowPipelineJob mockJob = mock(DataflowPipelineJob.class);
+ when(mockJob.getProjectId()).thenReturn(projectId);
+ when(mockJob.getJobId()).thenReturn(jobId);
+ when(mockJob.waitToFinish(
+ anyLong(), isA(TimeUnit.class), isA(MonitoringUtil.JobMessagesHandler.class)))
+ .thenReturn(terminalState);
+ return mockJob;
+ }
+
+ /**
+ * Returns a {@link BlockingDataflowPipelineRunner} that will return the provided a job to return.
+ * Some {@link PipelineOptions} will be extracted from the job, such as the project ID.
+ */
+ private BlockingDataflowPipelineRunner createMockRunner(DataflowPipelineJob job)
+ throws Exception {
+ DataflowPipelineRunner mockRunner = mock(DataflowPipelineRunner.class);
+ TestDataflowPipelineOptions options =
+ PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
+ options.setProject(job.getProjectId());
+
+ when(mockRunner.run(isA(Pipeline.class))).thenReturn(job);
+
+ return new BlockingDataflowPipelineRunner(mockRunner, options);
+ }
+
+ /**
+ * Tests that the {@link BlockingDataflowPipelineRunner} returns normally when a job terminates in
+ * the {@link State#DONE DONE} state.
+ */
+ @Test
+ public void testJobDoneComplete() throws Exception {
+ createMockRunner(createMockJob("testJobDone-projectId", "testJobDone-jobId", State.DONE))
+ .run(TestPipeline.create());
+ expectedLogs.verifyInfo("Job finished with status DONE");
+ }
+
+ /**
+ * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
+ * when a job terminates in the {@link State#FAILED FAILED} state.
+ */
+ @Test
+ public void testFailedJobThrowsException() throws Exception {
+ expectedThrown.expect(DataflowJobExecutionException.class);
+ expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+ JobIdMatcher.expectJobId("testFailedJob-jobId")));
+ createMockRunner(createMockJob("testFailedJob-projectId", "testFailedJob-jobId", State.FAILED))
+ .run(TestPipeline.create());
+ }
+
+ /**
+ * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
+ * when a job terminates in the {@link State#CANCELLED CANCELLED} state.
+ */
+ @Test
+ public void testCancelledJobThrowsException() throws Exception {
+ expectedThrown.expect(DataflowJobCancelledException.class);
+ expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+ JobIdMatcher.expectJobId("testCancelledJob-jobId")));
+ createMockRunner(
+ createMockJob("testCancelledJob-projectId", "testCancelledJob-jobId", State.CANCELLED))
+ .run(TestPipeline.create());
+ }
+
+ /**
+ * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
+ * when a job terminates in the {@link State#UPDATED UPDATED} state.
+ */
+ @Test
+ public void testUpdatedJobThrowsException() throws Exception {
+ expectedThrown.expect(DataflowJobUpdatedException.class);
+ expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+ JobIdMatcher.expectJobId("testUpdatedJob-jobId")));
+ expectedThrown.expect(ReplacedByJobMatcher.expectReplacedBy(
+ JobIdMatcher.expectJobId("testUpdatedJob-replacedByJobId")));
+ DataflowPipelineJob job =
+ createMockJob("testUpdatedJob-projectId", "testUpdatedJob-jobId", State.UPDATED);
+ DataflowPipelineJob replacedByJob =
+ createMockJob("testUpdatedJob-projectId", "testUpdatedJob-replacedByJobId", State.DONE);
+ when(job.getReplacedByJob()).thenReturn(replacedByJob);
+ createMockRunner(job).run(TestPipeline.create());
+ }
+
+ /**
+ * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
+ * when a job terminates in the {@link State#UNKNOWN UNKNOWN} state, indicating that the
+ * Dataflow service returned a state that the SDK is unfamiliar with (possibly because it
+ * is an old SDK relative the service).
+ */
+ @Test
+ public void testUnknownJobThrowsException() throws Exception {
+ expectedThrown.expect(IllegalStateException.class);
+ createMockRunner(
+ createMockJob("testUnknownJob-projectId", "testUnknownJob-jobId", State.UNKNOWN))
+ .run(TestPipeline.create());
+ }
+
+ /**
+ * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
+ * when a job returns a {@code null} state, indicating that it failed to contact the service,
+ * including all of its built-in resilience logic.
+ */
+ @Test
+ public void testNullJobThrowsException() throws Exception {
+ expectedThrown.expect(DataflowServiceException.class);
+ expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
+ JobIdMatcher.expectJobId("testNullJob-jobId")));
+ createMockRunner(createMockJob("testNullJob-projectId", "testNullJob-jobId", null))
+ .run(TestPipeline.create());
+ }
+
+ @Test
+ public void testToString() {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setJobName("TestJobName");
+ options.setProject("test-project");
+ options.setTempLocation("gs://test/temp/location");
+ options.setGcpCredential(new TestCredential());
+ options.setPathValidatorClass(NoopPathValidator.class);
+ assertEquals("BlockingDataflowPipelineRunner#testjobname",
+ BlockingDataflowPipelineRunner.fromOptions(options).toString());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
new file mode 100644
index 0000000..80b7e7b
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
@@ -0,0 +1,608 @@
+/*
+ * 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.dataflow;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.hasEntry;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThanOrEqualTo;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.runners.AggregatorRetrievalException;
+import org.apache.beam.sdk.runners.AggregatorValues;
+import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get;
+import com.google.api.services.dataflow.Dataflow.Projects.Jobs.GetMetrics;
+import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.JobMetrics;
+import com.google.api.services.dataflow.model.MetricStructuredName;
+import com.google.api.services.dataflow.model.MetricUpdate;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSetMultimap;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.net.SocketTimeoutException;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests for DataflowPipelineJob.
+ */
+@RunWith(JUnit4.class)
+public class DataflowPipelineJobTest {
+ private static final String PROJECT_ID = "someProject";
+ private static final String JOB_ID = "1234";
+
+ @Mock
+ private Dataflow mockWorkflowClient;
+ @Mock
+ private Dataflow.Projects mockProjects;
+ @Mock
+ private Dataflow.Projects.Jobs mockJobs;
+ @Rule
+ public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper();
+
+ @Rule
+ public ExpectedException thrown = ExpectedException.none();
+
+ @Before
+ public void setup() {
+ MockitoAnnotations.initMocks(this);
+
+ when(mockWorkflowClient.projects()).thenReturn(mockProjects);
+ when(mockProjects.jobs()).thenReturn(mockJobs);
+ }
+
+ /**
+ * Validates that a given time is valid for the total time slept by a
+ * AttemptBoundedExponentialBackOff given the number of retries and
+ * an initial polling interval.
+ *
+ * @param pollingIntervalMillis The initial polling interval given.
+ * @param attempts The number of attempts made
+ * @param timeSleptMillis The amount of time slept by the clock. This is checked
+ * against the valid interval.
+ */
+ void checkValidInterval(long pollingIntervalMillis, int attempts, long timeSleptMillis) {
+ long highSum = 0;
+ long lowSum = 0;
+ for (int i = 1; i < attempts; i++) {
+ double currentInterval =
+ pollingIntervalMillis
+ * Math.pow(AttemptBoundedExponentialBackOff.DEFAULT_MULTIPLIER, i - 1);
+ double offset =
+ AttemptBoundedExponentialBackOff.DEFAULT_RANDOMIZATION_FACTOR * currentInterval;
+ highSum += Math.round(currentInterval + offset);
+ lowSum += Math.round(currentInterval - offset);
+ }
+ assertThat(timeSleptMillis, allOf(greaterThanOrEqualTo(lowSum), lessThanOrEqualTo(highSum)));
+ }
+
+ @Test
+ public void testWaitToFinishMessagesFail() throws Exception {
+ Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+ Job statusResponse = new Job();
+ statusResponse.setCurrentState("JOB_STATE_" + State.DONE.name());
+ when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+ when(statusRequest.execute()).thenReturn(statusResponse);
+
+ MonitoringUtil.JobMessagesHandler jobHandler = mock(MonitoringUtil.JobMessagesHandler.class);
+ Dataflow.Projects.Jobs.Messages mockMessages =
+ mock(Dataflow.Projects.Jobs.Messages.class);
+ Messages.List listRequest = mock(Dataflow.Projects.Jobs.Messages.List.class);
+ when(mockJobs.messages()).thenReturn(mockMessages);
+ when(mockMessages.list(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(listRequest);
+ when(listRequest.execute()).thenThrow(SocketTimeoutException.class);
+ DataflowAggregatorTransforms dataflowAggregatorTransforms =
+ mock(DataflowAggregatorTransforms.class);
+
+ DataflowPipelineJob job = new DataflowPipelineJob(
+ PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+
+ State state = job.waitToFinish(5, TimeUnit.MINUTES, jobHandler, fastClock, fastClock);
+ assertEquals(null, state);
+ }
+
+ public State mockWaitToFinishInState(State state) throws Exception {
+ Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+ Job statusResponse = new Job();
+ statusResponse.setCurrentState("JOB_STATE_" + state.name());
+
+ when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+ when(statusRequest.execute()).thenReturn(statusResponse);
+ DataflowAggregatorTransforms dataflowAggregatorTransforms =
+ mock(DataflowAggregatorTransforms.class);
+
+ DataflowPipelineJob job = new DataflowPipelineJob(
+ PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+
+ return job.waitToFinish(1, TimeUnit.MINUTES, null, fastClock, fastClock);
+ }
+
+ /**
+ * Tests that the {@link DataflowPipelineJob} understands that the {@link State#DONE DONE}
+ * state is terminal.
+ */
+ @Test
+ public void testWaitToFinishDone() throws Exception {
+ assertEquals(State.DONE, mockWaitToFinishInState(State.DONE));
+ }
+
+ /**
+ * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
+ * state is terminal.
+ */
+ @Test
+ public void testWaitToFinishFailed() throws Exception {
+ assertEquals(State.FAILED, mockWaitToFinishInState(State.FAILED));
+ }
+
+ /**
+ * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
+ * state is terminal.
+ */
+ @Test
+ public void testWaitToFinishCancelled() throws Exception {
+ assertEquals(State.CANCELLED, mockWaitToFinishInState(State.CANCELLED));
+ }
+
+ /**
+ * Tests that the {@link DataflowPipelineJob} understands that the {@link State#FAILED FAILED}
+ * state is terminal.
+ */
+ @Test
+ public void testWaitToFinishUpdated() throws Exception {
+ assertEquals(State.UPDATED, mockWaitToFinishInState(State.UPDATED));
+ }
+
+ @Test
+ public void testWaitToFinishFail() throws Exception {
+ Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+ when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+ when(statusRequest.execute()).thenThrow(IOException.class);
+ DataflowAggregatorTransforms dataflowAggregatorTransforms =
+ mock(DataflowAggregatorTransforms.class);
+
+ DataflowPipelineJob job = new DataflowPipelineJob(
+ PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+
+ long startTime = fastClock.nanoTime();
+ State state = job.waitToFinish(5, TimeUnit.MINUTES, null, fastClock, fastClock);
+ assertEquals(null, state);
+ long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
+ checkValidInterval(DataflowPipelineJob.MESSAGES_POLLING_INTERVAL,
+ DataflowPipelineJob.MESSAGES_POLLING_ATTEMPTS, timeDiff);
+ }
+
+ @Test
+ public void testWaitToFinishTimeFail() throws Exception {
+ Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+ when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+ when(statusRequest.execute()).thenThrow(IOException.class);
+ DataflowAggregatorTransforms dataflowAggregatorTransforms =
+ mock(DataflowAggregatorTransforms.class);
+
+ DataflowPipelineJob job = new DataflowPipelineJob(
+ PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+ long startTime = fastClock.nanoTime();
+ State state = job.waitToFinish(4, TimeUnit.MILLISECONDS, null, fastClock, fastClock);
+ assertEquals(null, state);
+ long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
+ // Should only sleep for the 4 ms remaining.
+ assertEquals(timeDiff, 4L);
+ }
+
+ @Test
+ public void testGetStateReturnsServiceState() throws Exception {
+ Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+ Job statusResponse = new Job();
+ statusResponse.setCurrentState("JOB_STATE_" + State.RUNNING.name());
+
+ when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+ when(statusRequest.execute()).thenReturn(statusResponse);
+
+ DataflowAggregatorTransforms dataflowAggregatorTransforms =
+ mock(DataflowAggregatorTransforms.class);
+
+ DataflowPipelineJob job = new DataflowPipelineJob(
+ PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+
+ assertEquals(
+ State.RUNNING,
+ job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock));
+ }
+
+ @Test
+ public void testGetStateWithExceptionReturnsUnknown() throws Exception {
+ Dataflow.Projects.Jobs.Get statusRequest = mock(Dataflow.Projects.Jobs.Get.class);
+
+ when(mockJobs.get(eq(PROJECT_ID), eq(JOB_ID))).thenReturn(statusRequest);
+ when(statusRequest.execute()).thenThrow(IOException.class);
+ DataflowAggregatorTransforms dataflowAggregatorTransforms =
+ mock(DataflowAggregatorTransforms.class);
+
+ DataflowPipelineJob job = new DataflowPipelineJob(
+ PROJECT_ID, JOB_ID, mockWorkflowClient, dataflowAggregatorTransforms);
+
+ long startTime = fastClock.nanoTime();
+ assertEquals(
+ State.UNKNOWN,
+ job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock));
+ long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime);
+ checkValidInterval(DataflowPipelineJob.STATUS_POLLING_INTERVAL,
+ DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, timeDiff);
+ }
+
+ @Test
+ public void testGetAggregatorValuesWithNoMetricUpdatesReturnsEmptyValue()
+ throws IOException, AggregatorRetrievalException {
+ Aggregator<?, ?> aggregator = mock(Aggregator.class);
+ @SuppressWarnings("unchecked")
+ PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+ String stepName = "s1";
+ String fullName = "Foo/Bar/Baz";
+ AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+ DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+ ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
+ ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
+
+ GetMetrics getMetrics = mock(GetMetrics.class);
+ when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+ JobMetrics jobMetrics = new JobMetrics();
+ when(getMetrics.execute()).thenReturn(jobMetrics);
+
+ jobMetrics.setMetrics(ImmutableList.<MetricUpdate>of());
+
+ Get getState = mock(Get.class);
+ when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+ Job modelJob = new Job();
+ when(getState.execute()).thenReturn(modelJob);
+ modelJob.setCurrentState(State.RUNNING.toString());
+
+ DataflowPipelineJob job =
+ new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+ AggregatorValues<?> values = job.getAggregatorValues(aggregator);
+
+ assertThat(values.getValues(), empty());
+ }
+
+ @Test
+ public void testGetAggregatorValuesWithNullMetricUpdatesReturnsEmptyValue()
+ throws IOException, AggregatorRetrievalException {
+ Aggregator<?, ?> aggregator = mock(Aggregator.class);
+ @SuppressWarnings("unchecked")
+ PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+ String stepName = "s1";
+ String fullName = "Foo/Bar/Baz";
+ AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+ DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+ ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
+ ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
+
+ GetMetrics getMetrics = mock(GetMetrics.class);
+ when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+ JobMetrics jobMetrics = new JobMetrics();
+ when(getMetrics.execute()).thenReturn(jobMetrics);
+
+ jobMetrics.setMetrics(null);
+
+ Get getState = mock(Get.class);
+ when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+ Job modelJob = new Job();
+ when(getState.execute()).thenReturn(modelJob);
+ modelJob.setCurrentState(State.RUNNING.toString());
+
+ DataflowPipelineJob job =
+ new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+ AggregatorValues<?> values = job.getAggregatorValues(aggregator);
+
+ assertThat(values.getValues(), empty());
+ }
+
+ @Test
+ public void testGetAggregatorValuesWithSingleMetricUpdateReturnsSingletonCollection()
+ throws IOException, AggregatorRetrievalException {
+ CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
+ String aggregatorName = "agg";
+ Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
+ @SuppressWarnings("unchecked")
+ PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+ String stepName = "s1";
+ String fullName = "Foo/Bar/Baz";
+ AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+ DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+ ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
+ ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
+
+ GetMetrics getMetrics = mock(GetMetrics.class);
+ when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+ JobMetrics jobMetrics = new JobMetrics();
+ when(getMetrics.execute()).thenReturn(jobMetrics);
+
+ MetricUpdate update = new MetricUpdate();
+ long stepValue = 1234L;
+ update.setScalar(new BigDecimal(stepValue));
+
+ MetricStructuredName structuredName = new MetricStructuredName();
+ structuredName.setName(aggregatorName);
+ structuredName.setContext(ImmutableMap.of("step", stepName));
+ update.setName(structuredName);
+
+ jobMetrics.setMetrics(ImmutableList.of(update));
+
+ Get getState = mock(Get.class);
+ when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+ Job modelJob = new Job();
+ when(getState.execute()).thenReturn(modelJob);
+ modelJob.setCurrentState(State.RUNNING.toString());
+
+ DataflowPipelineJob job =
+ new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+ AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
+
+ assertThat(values.getValuesAtSteps(), hasEntry(fullName, stepValue));
+ assertThat(values.getValuesAtSteps().size(), equalTo(1));
+ assertThat(values.getValues(), contains(stepValue));
+ assertThat(values.getTotalValue(combineFn), equalTo(Long.valueOf(stepValue)));
+ }
+
+ @Test
+ public void testGetAggregatorValuesWithMultipleMetricUpdatesReturnsCollection()
+ throws IOException, AggregatorRetrievalException {
+ CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
+ String aggregatorName = "agg";
+ Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
+
+ @SuppressWarnings("unchecked")
+ PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+ String stepName = "s1";
+ String fullName = "Foo/Bar/Baz";
+ AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+ @SuppressWarnings("unchecked")
+ PTransform<PInput, POutput> otherTransform = mock(PTransform.class);
+ String otherStepName = "s88";
+ String otherFullName = "Spam/Ham/Eggs";
+ AppliedPTransform<?, ?, ?> otherAppliedTransform =
+ appliedPTransform(otherFullName, otherTransform);
+
+ DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+ ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(
+ aggregator, pTransform, aggregator, otherTransform).asMap(),
+ ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(
+ appliedTransform, stepName, otherAppliedTransform, otherStepName));
+
+ GetMetrics getMetrics = mock(GetMetrics.class);
+ when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+ JobMetrics jobMetrics = new JobMetrics();
+ when(getMetrics.execute()).thenReturn(jobMetrics);
+
+ MetricUpdate updateOne = new MetricUpdate();
+ long stepValue = 1234L;
+ updateOne.setScalar(new BigDecimal(stepValue));
+
+ MetricStructuredName structuredNameOne = new MetricStructuredName();
+ structuredNameOne.setName(aggregatorName);
+ structuredNameOne.setContext(ImmutableMap.of("step", stepName));
+ updateOne.setName(structuredNameOne);
+
+ MetricUpdate updateTwo = new MetricUpdate();
+ long stepValueTwo = 1024L;
+ updateTwo.setScalar(new BigDecimal(stepValueTwo));
+
+ MetricStructuredName structuredNameTwo = new MetricStructuredName();
+ structuredNameTwo.setName(aggregatorName);
+ structuredNameTwo.setContext(ImmutableMap.of("step", otherStepName));
+ updateTwo.setName(structuredNameTwo);
+
+ jobMetrics.setMetrics(ImmutableList.of(updateOne, updateTwo));
+
+ Get getState = mock(Get.class);
+ when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+ Job modelJob = new Job();
+ when(getState.execute()).thenReturn(modelJob);
+ modelJob.setCurrentState(State.RUNNING.toString());
+
+ DataflowPipelineJob job =
+ new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+ AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
+
+ assertThat(values.getValuesAtSteps(), hasEntry(fullName, stepValue));
+ assertThat(values.getValuesAtSteps(), hasEntry(otherFullName, stepValueTwo));
+ assertThat(values.getValuesAtSteps().size(), equalTo(2));
+ assertThat(values.getValues(), containsInAnyOrder(stepValue, stepValueTwo));
+ assertThat(values.getTotalValue(combineFn), equalTo(Long.valueOf(stepValue + stepValueTwo)));
+ }
+
+ @Test
+ public void testGetAggregatorValuesWithUnrelatedMetricUpdateIgnoresUpdate()
+ throws IOException, AggregatorRetrievalException {
+ CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
+ String aggregatorName = "agg";
+ Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
+ @SuppressWarnings("unchecked")
+ PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+ String stepName = "s1";
+ String fullName = "Foo/Bar/Baz";
+ AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+ DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+ ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
+ ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
+
+ GetMetrics getMetrics = mock(GetMetrics.class);
+ when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+ JobMetrics jobMetrics = new JobMetrics();
+ when(getMetrics.execute()).thenReturn(jobMetrics);
+
+ MetricUpdate ignoredUpdate = new MetricUpdate();
+ ignoredUpdate.setScalar(null);
+
+ MetricStructuredName ignoredName = new MetricStructuredName();
+ ignoredName.setName("ignoredAggregator.elementCount.out0");
+ ignoredName.setContext(null);
+ ignoredUpdate.setName(ignoredName);
+
+ jobMetrics.setMetrics(ImmutableList.of(ignoredUpdate));
+
+ Get getState = mock(Get.class);
+ when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+ Job modelJob = new Job();
+ when(getState.execute()).thenReturn(modelJob);
+ modelJob.setCurrentState(State.RUNNING.toString());
+
+ DataflowPipelineJob job =
+ new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+ AggregatorValues<Long> values = job.getAggregatorValues(aggregator);
+
+ assertThat(values.getValuesAtSteps().entrySet(), empty());
+ assertThat(values.getValues(), empty());
+ }
+
+ @Test
+ public void testGetAggregatorValuesWithUnusedAggregatorThrowsException()
+ throws AggregatorRetrievalException {
+ Aggregator<?, ?> aggregator = mock(Aggregator.class);
+
+ DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+ ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of().asMap(),
+ ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of());
+
+ DataflowPipelineJob job =
+ new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+ thrown.expect(IllegalArgumentException.class);
+ thrown.expectMessage("not used in this pipeline");
+
+ job.getAggregatorValues(aggregator);
+ }
+
+ @Test
+ public void testGetAggregatorValuesWhenClientThrowsExceptionThrowsAggregatorRetrievalException()
+ throws IOException, AggregatorRetrievalException {
+ CombineFn<Long, long[], Long> combineFn = new Sum.SumLongFn();
+ String aggregatorName = "agg";
+ Aggregator<Long, Long> aggregator = new TestAggregator<>(combineFn, aggregatorName);
+ @SuppressWarnings("unchecked")
+ PTransform<PInput, POutput> pTransform = mock(PTransform.class);
+ String stepName = "s1";
+ String fullName = "Foo/Bar/Baz";
+ AppliedPTransform<?, ?, ?> appliedTransform = appliedPTransform(fullName, pTransform);
+
+ DataflowAggregatorTransforms aggregatorTransforms = new DataflowAggregatorTransforms(
+ ImmutableSetMultimap.<Aggregator<?, ?>, PTransform<?, ?>>of(aggregator, pTransform).asMap(),
+ ImmutableMap.<AppliedPTransform<?, ?, ?>, String>of(appliedTransform, stepName));
+
+ GetMetrics getMetrics = mock(GetMetrics.class);
+ when(mockJobs.getMetrics(PROJECT_ID, JOB_ID)).thenReturn(getMetrics);
+ IOException cause = new IOException();
+ when(getMetrics.execute()).thenThrow(cause);
+
+ Get getState = mock(Get.class);
+ when(mockJobs.get(PROJECT_ID, JOB_ID)).thenReturn(getState);
+ Job modelJob = new Job();
+ when(getState.execute()).thenReturn(modelJob);
+ modelJob.setCurrentState(State.RUNNING.toString());
+
+ DataflowPipelineJob job =
+ new DataflowPipelineJob(PROJECT_ID, JOB_ID, mockWorkflowClient, aggregatorTransforms);
+
+ thrown.expect(AggregatorRetrievalException.class);
+ thrown.expectCause(is(cause));
+ thrown.expectMessage(aggregator.toString());
+ thrown.expectMessage("when retrieving Aggregator values for");
+
+ job.getAggregatorValues(aggregator);
+ }
+
+ private static class TestAggregator<InT, OutT> implements Aggregator<InT, OutT> {
+ private final CombineFn<InT, ?, OutT> combineFn;
+ private final String name;
+
+ public TestAggregator(CombineFn<InT, ?, OutT> combineFn, String name) {
+ this.combineFn = combineFn;
+ this.name = name;
+ }
+
+ @Override
+ public void addValue(InT value) {
+ throw new AssertionError();
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public CombineFn<InT, ?, OutT> getCombineFn() {
+ return combineFn;
+ }
+ }
+
+ private AppliedPTransform<?, ?, ?> appliedPTransform(
+ String fullName, PTransform<PInput, POutput> transform) {
+ return AppliedPTransform.of(fullName, mock(PInput.class), mock(POutput.class), transform);
+ }
+}
[06/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java
new file mode 100644
index 0000000..f0e677e
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.dataflow.transforms;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+import org.joda.time.Duration;
+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.Arrays;
+import java.util.List;
+
+/** Tests for {@link GroupByKey} for the {@link DataflowPipelineRunner}. */
+@RunWith(JUnit4.class)
+public class DataflowGroupByKeyTest {
+ @Rule
+ public ExpectedException thrown = ExpectedException.none();
+
+ /**
+ * Create a test pipeline that uses the {@link DataflowPipelineRunner} so that {@link GroupByKey}
+ * is not expanded. This is used for verifying that even without expansion the proper errors show
+ * up.
+ */
+ private Pipeline createTestServiceRunner() {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setProject("someproject");
+ options.setStagingLocation("gs://staging");
+ options.setPathValidatorClass(NoopPathValidator.class);
+ options.setDataflowClient(null);
+ return Pipeline.create(options);
+ }
+
+ @Test
+ public void testInvalidWindowsService() {
+ Pipeline p = createTestServiceRunner();
+
+ List<KV<String, Integer>> ungroupedPairs = Arrays.asList();
+
+ PCollection<KV<String, Integer>> input =
+ p.apply(Create.of(ungroupedPairs)
+ .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())))
+ .apply(Window.<KV<String, Integer>>into(
+ Sessions.withGapDuration(Duration.standardMinutes(1))));
+
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("GroupByKey must have a valid Window merge function");
+ input
+ .apply("GroupByKey", GroupByKey.<String, Integer>create())
+ .apply("GroupByKeyAgain", GroupByKey.<String, Iterable<Integer>>create());
+ }
+
+ @Test
+ public void testGroupByKeyServiceUnbounded() {
+ Pipeline p = createTestServiceRunner();
+
+ PCollection<KV<String, Integer>> input =
+ p.apply(
+ new PTransform<PBegin, PCollection<KV<String, Integer>>>() {
+ @Override
+ public PCollection<KV<String, Integer>> apply(PBegin input) {
+ return PCollection.<KV<String, Integer>>createPrimitiveOutputInternal(
+ input.getPipeline(),
+ WindowingStrategy.globalDefault(),
+ PCollection.IsBounded.UNBOUNDED)
+ .setTypeDescriptorInternal(new TypeDescriptor<KV<String, Integer>>() {});
+ }
+ });
+
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage(
+ "GroupByKey cannot be applied to non-bounded PCollection in the GlobalWindow without "
+ + "a trigger. Use a Window.into or Window.triggering transform prior to GroupByKey.");
+
+ input.apply("GroupByKey", GroupByKey.<String, Integer>create());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java
new file mode 100644
index 0000000..d787500
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java
@@ -0,0 +1,208 @@
+/*
+ * 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.dataflow.transforms;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.InvalidWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.NoopPathValidator;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Duration;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.internal.matchers.ThrowableMessageMatcher;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link View} for a {@link DataflowPipelineRunner}. */
+@RunWith(JUnit4.class)
+public class DataflowViewTest {
+ @Rule
+ public transient ExpectedException thrown = ExpectedException.none();
+
+ private Pipeline createTestBatchRunner() {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setProject("someproject");
+ options.setStagingLocation("gs://staging");
+ options.setPathValidatorClass(NoopPathValidator.class);
+ options.setDataflowClient(null);
+ return Pipeline.create(options);
+ }
+
+ private Pipeline createTestStreamingRunner() {
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setStreaming(true);
+ options.setProject("someproject");
+ options.setStagingLocation("gs://staging");
+ options.setPathValidatorClass(NoopPathValidator.class);
+ options.setDataflowClient(null);
+ return Pipeline.create(options);
+ }
+
+ private void testViewUnbounded(
+ Pipeline pipeline,
+ PTransform<PCollection<KV<String, Integer>>, ? extends PCollectionView<?>> view) {
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("Unable to create a side-input view from input");
+ thrown.expectCause(
+ ThrowableMessageMatcher.hasMessage(Matchers.containsString("non-bounded PCollection")));
+ pipeline
+ .apply(
+ new PTransform<PBegin, PCollection<KV<String, Integer>>>() {
+ @Override
+ public PCollection<KV<String, Integer>> apply(PBegin input) {
+ return PCollection.<KV<String, Integer>>createPrimitiveOutputInternal(
+ input.getPipeline(),
+ WindowingStrategy.globalDefault(),
+ PCollection.IsBounded.UNBOUNDED)
+ .setTypeDescriptorInternal(new TypeDescriptor<KV<String, Integer>>() {});
+ }
+ })
+ .apply(view);
+ }
+
+ private void testViewNonmerging(
+ Pipeline pipeline,
+ PTransform<PCollection<KV<String, Integer>>, ? extends PCollectionView<?>> view) {
+ thrown.expect(IllegalStateException.class);
+ thrown.expectMessage("Unable to create a side-input view from input");
+ thrown.expectCause(
+ ThrowableMessageMatcher.hasMessage(Matchers.containsString("Consumed by GroupByKey")));
+ pipeline.apply(Create.<KV<String, Integer>>of(KV.of("hello", 5)))
+ .apply(Window.<KV<String, Integer>>into(new InvalidWindows<>(
+ "Consumed by GroupByKey", FixedWindows.of(Duration.standardHours(1)))))
+ .apply(view);
+ }
+
+ @Test
+ public void testViewUnboundedAsSingletonBatch() {
+ testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asSingleton());
+ }
+
+ @Test
+ public void testViewUnboundedAsSingletonStreaming() {
+ testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asSingleton());
+ }
+
+ @Test
+ public void testViewUnboundedAsIterableBatch() {
+ testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asIterable());
+ }
+
+ @Test
+ public void testViewUnboundedAsIterableStreaming() {
+ testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asIterable());
+ }
+
+ @Test
+ public void testViewUnboundedAsListBatch() {
+ testViewUnbounded(createTestBatchRunner(), View.<KV<String, Integer>>asList());
+ }
+
+ @Test
+ public void testViewUnboundedAsListStreaming() {
+ testViewUnbounded(createTestStreamingRunner(), View.<KV<String, Integer>>asList());
+ }
+
+ @Test
+ public void testViewUnboundedAsMapBatch() {
+ testViewUnbounded(createTestBatchRunner(), View.<String, Integer>asMap());
+ }
+
+ @Test
+ public void testViewUnboundedAsMapStreaming() {
+ testViewUnbounded(createTestStreamingRunner(), View.<String, Integer>asMap());
+ }
+
+ @Test
+ public void testViewUnboundedAsMultimapBatch() {
+ testViewUnbounded(createTestBatchRunner(), View.<String, Integer>asMultimap());
+ }
+
+ @Test
+ public void testViewUnboundedAsMultimapStreaming() {
+ testViewUnbounded(createTestStreamingRunner(), View.<String, Integer>asMultimap());
+ }
+
+ @Test
+ public void testViewNonmergingAsSingletonBatch() {
+ testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asSingleton());
+ }
+
+ @Test
+ public void testViewNonmergingAsSingletonStreaming() {
+ testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asSingleton());
+ }
+
+ @Test
+ public void testViewNonmergingAsIterableBatch() {
+ testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asIterable());
+ }
+
+ @Test
+ public void testViewNonmergingAsIterableStreaming() {
+ testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asIterable());
+ }
+
+ @Test
+ public void testViewNonmergingAsListBatch() {
+ testViewNonmerging(createTestBatchRunner(), View.<KV<String, Integer>>asList());
+ }
+
+ @Test
+ public void testViewNonmergingAsListStreaming() {
+ testViewNonmerging(createTestStreamingRunner(), View.<KV<String, Integer>>asList());
+ }
+
+ @Test
+ public void testViewNonmergingAsMapBatch() {
+ testViewNonmerging(createTestBatchRunner(), View.<String, Integer>asMap());
+ }
+
+ @Test
+ public void testViewNonmergingAsMapStreaming() {
+ testViewNonmerging(createTestStreamingRunner(), View.<String, Integer>asMap());
+ }
+
+ @Test
+ public void testViewNonmergingAsMultimapBatch() {
+ testViewNonmerging(createTestBatchRunner(), View.<String, Integer>asMultimap());
+ }
+
+ @Test
+ public void testViewNonmergingAsMultimapStreaming() {
+ testViewNonmerging(createTestStreamingRunner(), View.<String, Integer>asMultimap());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/DataflowPathValidatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/DataflowPathValidatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/DataflowPathValidatorTest.java
new file mode 100644
index 0000000..5587986
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/DataflowPathValidatorTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.dataflow.util;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+/** Tests for {@link DataflowPathValidator}. */
+@RunWith(JUnit4.class)
+public class DataflowPathValidatorTest {
+ @Rule public ExpectedException expectedException = ExpectedException.none();
+
+ @Mock private GcsUtil mockGcsUtil;
+ private DataflowPathValidator validator;
+
+ @Before
+ public void setUp() throws Exception {
+ MockitoAnnotations.initMocks(this);
+ when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(true);
+ when(mockGcsUtil.isGcsPatternSupported(anyString())).thenCallRealMethod();
+ DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
+ options.setGcpCredential(new TestCredential());
+ options.setRunner(DataflowPipelineRunner.class);
+ options.setGcsUtil(mockGcsUtil);
+ validator = new DataflowPathValidator(options);
+ }
+
+ @Test
+ public void testValidFilePattern() {
+ validator.validateInputFilePatternSupported("gs://bucket/path");
+ }
+
+ @Test
+ public void testInvalidFilePattern() {
+ expectedException.expect(IllegalArgumentException.class);
+ expectedException.expectMessage(
+ "DataflowPipelineRunner expected a valid 'gs://' path but was given '/local/path'");
+ validator.validateInputFilePatternSupported("/local/path");
+ }
+
+ @Test
+ public void testWhenBucketDoesNotExist() throws Exception {
+ when(mockGcsUtil.bucketExists(any(GcsPath.class))).thenReturn(false);
+ expectedException.expect(IllegalArgumentException.class);
+ expectedException.expectMessage(
+ "Could not find file gs://non-existent-bucket/location");
+ validator.validateInputFilePatternSupported("gs://non-existent-bucket/location");
+ }
+
+ @Test
+ public void testValidOutputPrefix() {
+ validator.validateOutputFilePrefixSupported("gs://bucket/path");
+ }
+
+ @Test
+ public void testInvalidOutputPrefix() {
+ expectedException.expect(IllegalArgumentException.class);
+ expectedException.expectMessage(
+ "DataflowPipelineRunner expected a valid 'gs://' path but was given '/local/path'");
+ validator.validateOutputFilePrefixSupported("/local/path");
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java
new file mode 100644
index 0000000..ee1532d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.dataflow.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.TestCredential;
+import org.apache.beam.sdk.util.TimeUtil;
+
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.JobMessage;
+import com.google.api.services.dataflow.model.ListJobMessagesResponse;
+
+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.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for MonitoringUtil.
+ */
+@RunWith(JUnit4.class)
+public class MonitoringUtilTest {
+ private static final String PROJECT_ID = "someProject";
+ private static final String JOB_ID = "1234";
+
+ @Rule public ExpectedException thrown = ExpectedException.none();
+
+ @Test
+ public void testGetJobMessages() throws IOException {
+ Dataflow.Projects.Jobs.Messages mockMessages = mock(Dataflow.Projects.Jobs.Messages.class);
+
+ // Two requests are needed to get all the messages.
+ Dataflow.Projects.Jobs.Messages.List firstRequest =
+ mock(Dataflow.Projects.Jobs.Messages.List.class);
+ Dataflow.Projects.Jobs.Messages.List secondRequest =
+ mock(Dataflow.Projects.Jobs.Messages.List.class);
+
+ when(mockMessages.list(PROJECT_ID, JOB_ID)).thenReturn(firstRequest).thenReturn(secondRequest);
+
+ ListJobMessagesResponse firstResponse = new ListJobMessagesResponse();
+ firstResponse.setJobMessages(new ArrayList<JobMessage>());
+ for (int i = 0; i < 100; ++i) {
+ JobMessage message = new JobMessage();
+ message.setId("message_" + i);
+ message.setTime(TimeUtil.toCloudTime(new Instant(i)));
+ firstResponse.getJobMessages().add(message);
+ }
+ String pageToken = "page_token";
+ firstResponse.setNextPageToken(pageToken);
+
+ ListJobMessagesResponse secondResponse = new ListJobMessagesResponse();
+ secondResponse.setJobMessages(new ArrayList<JobMessage>());
+ for (int i = 100; i < 150; ++i) {
+ JobMessage message = new JobMessage();
+ message.setId("message_" + i);
+ message.setTime(TimeUtil.toCloudTime(new Instant(i)));
+ secondResponse.getJobMessages().add(message);
+ }
+
+ when(firstRequest.execute()).thenReturn(firstResponse);
+ when(secondRequest.execute()).thenReturn(secondResponse);
+
+ MonitoringUtil util = new MonitoringUtil(PROJECT_ID, mockMessages);
+
+ List<JobMessage> messages = util.getJobMessages(JOB_ID, -1);
+
+ verify(secondRequest).setPageToken(pageToken);
+
+ assertEquals(150, messages.size());
+ }
+
+ @Test
+ public void testToStateCreatesState() {
+ String stateName = "JOB_STATE_DONE";
+
+ State result = MonitoringUtil.toState(stateName);
+
+ assertEquals(State.DONE, result);
+ }
+
+ @Test
+ public void testToStateWithNullReturnsUnknown() {
+ String stateName = null;
+
+ State result = MonitoringUtil.toState(stateName);
+
+ assertEquals(State.UNKNOWN, result);
+ }
+
+ @Test
+ public void testToStateWithOtherValueReturnsUnknown() {
+ String stateName = "FOO_BAR_BAZ";
+
+ State result = MonitoringUtil.toState(stateName);
+
+ assertEquals(State.UNKNOWN, result);
+ }
+
+ @Test
+ public void testDontOverrideEndpointWithDefaultApi() {
+ DataflowPipelineOptions options =
+ PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
+ options.setProject(PROJECT_ID);
+ options.setGcpCredential(new TestCredential());
+ String cancelCommand = MonitoringUtil.getGcloudCancelCommand(options, JOB_ID);
+ assertEquals("gcloud alpha dataflow jobs --project=someProject cancel 1234", cancelCommand);
+ }
+
+ @Test
+ public void testOverridesEndpointWithStagedDataflowEndpoint() {
+ DataflowPipelineOptions options =
+ PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
+ options.setProject(PROJECT_ID);
+ options.setGcpCredential(new TestCredential());
+ String stagingDataflowEndpoint = "v0neverExisted";
+ options.setDataflowEndpoint(stagingDataflowEndpoint);
+ String cancelCommand = MonitoringUtil.getGcloudCancelCommand(options, JOB_ID);
+ assertEquals(
+ "CLOUDSDK_API_ENDPOINT_OVERRIDES_DATAFLOW=https://dataflow.googleapis.com/v0neverExisted/ "
+ + "gcloud alpha dataflow jobs --project=someProject cancel 1234",
+ cancelCommand);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java
new file mode 100644
index 0000000..41ad05d
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java
@@ -0,0 +1,486 @@
+/*
+ * 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.dataflow.util;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.runners.dataflow.util.PackageUtil.PackageAttributes;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.testing.FastNanoClockAndSleeper;
+import org.apache.beam.sdk.util.GcsUtil;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.gcsfs.GcsPath;
+
+import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo;
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.client.http.HttpRequest;
+import com.google.api.client.http.HttpResponse;
+import com.google.api.client.http.HttpStatusCodes;
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.http.LowLevelHttpRequest;
+import com.google.api.client.json.GenericJson;
+import com.google.api.client.json.Json;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.jackson2.JacksonFactory;
+import com.google.api.client.testing.http.HttpTesting;
+import com.google.api.client.testing.http.MockHttpTransport;
+import com.google.api.client.testing.http.MockLowLevelHttpRequest;
+import com.google.api.client.testing.http.MockLowLevelHttpResponse;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+import com.google.common.io.LineReader;
+
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matchers;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.channels.Channels;
+import java.nio.channels.Pipe;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/** Tests for PackageUtil. */
+@RunWith(JUnit4.class)
+public class PackageUtilTest {
+ @Rule public ExpectedLogs logged = ExpectedLogs.none(PackageUtil.class);
+ @Rule
+ public TemporaryFolder tmpFolder = new TemporaryFolder();
+
+ @Rule
+ public FastNanoClockAndSleeper fastNanoClockAndSleeper = new FastNanoClockAndSleeper();
+
+ @Mock
+ GcsUtil mockGcsUtil;
+
+ // 128 bits, base64 encoded is 171 bits, rounds to 22 bytes
+ private static final String HASH_PATTERN = "[a-zA-Z0-9+-]{22}";
+
+ // Hamcrest matcher to assert a string matches a pattern
+ private static class RegexMatcher extends BaseMatcher<String> {
+ private final Pattern pattern;
+
+ public RegexMatcher(String regex) {
+ this.pattern = Pattern.compile(regex);
+ }
+
+ @Override
+ public boolean matches(Object o) {
+ if (!(o instanceof String)) {
+ return false;
+ }
+ return pattern.matcher((String) o).matches();
+ }
+
+ @Override
+ public void describeTo(Description description) {
+ description.appendText(String.format("matches regular expression %s", pattern));
+ }
+
+ public static RegexMatcher matches(String regex) {
+ return new RegexMatcher(regex);
+ }
+ }
+
+ @Before
+ public void setUp() {
+ MockitoAnnotations.initMocks(this);
+
+ GcsOptions pipelineOptions = PipelineOptionsFactory.as(GcsOptions.class);
+ pipelineOptions.setGcsUtil(mockGcsUtil);
+
+ IOChannelUtils.registerStandardIOFactories(pipelineOptions);
+ }
+
+ private File makeFileWithContents(String name, String contents) throws Exception {
+ File tmpFile = tmpFolder.newFile(name);
+ Files.write(contents, tmpFile, StandardCharsets.UTF_8);
+ tmpFile.setLastModified(0); // required for determinism with directories
+ return tmpFile;
+ }
+
+ static final String STAGING_PATH = GcsPath.fromComponents("somebucket", "base/path").toString();
+ private static PackageAttributes makePackageAttributes(File file, String overridePackageName) {
+ return PackageUtil.createPackageAttributes(file, STAGING_PATH, overridePackageName);
+ }
+
+ @Test
+ public void testFileWithExtensionPackageNamingAndSize() throws Exception {
+ String contents = "This is a test!";
+ File tmpFile = makeFileWithContents("file.txt", contents);
+ PackageAttributes attr = makePackageAttributes(tmpFile, null);
+ DataflowPackage target = attr.getDataflowPackage();
+
+ assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN + ".txt"));
+ assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
+ assertThat(attr.getSize(), equalTo((long) contents.length()));
+ }
+
+ @Test
+ public void testPackageNamingWithFileNoExtension() throws Exception {
+ File tmpFile = makeFileWithContents("file", "This is a test!");
+ DataflowPackage target = makePackageAttributes(tmpFile, null).getDataflowPackage();
+
+ assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN));
+ assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
+ }
+
+ @Test
+ public void testPackageNamingWithDirectory() throws Exception {
+ File tmpDirectory = tmpFolder.newFolder("folder");
+ DataflowPackage target = makePackageAttributes(tmpDirectory, null).getDataflowPackage();
+
+ assertThat(target.getName(), RegexMatcher.matches("folder-" + HASH_PATTERN + ".jar"));
+ assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
+ }
+
+ @Test
+ public void testPackageNamingWithFilesHavingSameContentsAndSameNames() throws Exception {
+ File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
+ makeFileWithContents("folder1/folderA/sameName", "This is a test!");
+ DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
+
+ File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
+ makeFileWithContents("folder2/folderA/sameName", "This is a test!");
+ DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
+
+ assertEquals(target1.getName(), target2.getName());
+ assertEquals(target1.getLocation(), target2.getLocation());
+ }
+
+ @Test
+ public void testPackageNamingWithFilesHavingSameContentsButDifferentNames() throws Exception {
+ File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
+ makeFileWithContents("folder1/folderA/uniqueName1", "This is a test!");
+ DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
+
+ File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
+ makeFileWithContents("folder2/folderA/uniqueName2", "This is a test!");
+ DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
+
+ assertNotEquals(target1.getName(), target2.getName());
+ assertNotEquals(target1.getLocation(), target2.getLocation());
+ }
+
+ @Test
+ public void testPackageNamingWithDirectoriesHavingSameContentsButDifferentNames()
+ throws Exception {
+ File tmpDirectory1 = tmpFolder.newFolder("folder1", "folderA");
+ tmpFolder.newFolder("folder1", "folderA", "uniqueName1");
+ DataflowPackage target1 = makePackageAttributes(tmpDirectory1, null).getDataflowPackage();
+
+ File tmpDirectory2 = tmpFolder.newFolder("folder2", "folderA");
+ tmpFolder.newFolder("folder2", "folderA", "uniqueName2");
+ DataflowPackage target2 = makePackageAttributes(tmpDirectory2, null).getDataflowPackage();
+
+ assertNotEquals(target1.getName(), target2.getName());
+ assertNotEquals(target1.getLocation(), target2.getLocation());
+ }
+
+ @Test
+ public void testPackageUploadWithLargeClasspathLogsWarning() throws Exception {
+ File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+ // all files will be present and cached so no upload needed.
+ when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(tmpFile.length());
+
+ List<String> classpathElements = Lists.newLinkedList();
+ for (int i = 0; i < 1005; ++i) {
+ String eltName = "element" + i;
+ classpathElements.add(eltName + '=' + tmpFile.getAbsolutePath());
+ }
+
+ PackageUtil.stageClasspathElements(classpathElements, STAGING_PATH);
+
+ logged.verifyWarn("Your classpath contains 1005 elements, which Google Cloud Dataflow");
+ }
+
+ @Test
+ public void testPackageUploadWithFileSucceeds() throws Exception {
+ Pipe pipe = Pipe.open();
+ String contents = "This is a test!";
+ File tmpFile = makeFileWithContents("file.txt", contents);
+ when(mockGcsUtil.fileSize(any(GcsPath.class)))
+ .thenThrow(new FileNotFoundException("some/path"));
+ when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
+
+ List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
+ ImmutableList.of(tmpFile.getAbsolutePath()), STAGING_PATH);
+ DataflowPackage target = Iterables.getOnlyElement(targets);
+
+ verify(mockGcsUtil).fileSize(any(GcsPath.class));
+ verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+ verifyNoMoreInteractions(mockGcsUtil);
+
+ assertThat(target.getName(), RegexMatcher.matches("file-" + HASH_PATTERN + ".txt"));
+ assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
+ assertThat(new LineReader(Channels.newReader(pipe.source(), "UTF-8")).readLine(),
+ equalTo(contents));
+ }
+
+ @Test
+ public void testPackageUploadWithDirectorySucceeds() throws Exception {
+ Pipe pipe = Pipe.open();
+ File tmpDirectory = tmpFolder.newFolder("folder");
+ tmpFolder.newFolder("folder", "empty_directory");
+ tmpFolder.newFolder("folder", "directory");
+ makeFileWithContents("folder/file.txt", "This is a test!");
+ makeFileWithContents("folder/directory/file.txt", "This is also a test!");
+
+ when(mockGcsUtil.fileSize(any(GcsPath.class)))
+ .thenThrow(new FileNotFoundException("some/path"));
+ when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
+
+ PackageUtil.stageClasspathElements(
+ ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
+
+ verify(mockGcsUtil).fileSize(any(GcsPath.class));
+ verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+ verifyNoMoreInteractions(mockGcsUtil);
+
+ ZipInputStream inputStream = new ZipInputStream(Channels.newInputStream(pipe.source()));
+ List<String> zipEntryNames = new ArrayList<>();
+ for (ZipEntry entry = inputStream.getNextEntry(); entry != null;
+ entry = inputStream.getNextEntry()) {
+ zipEntryNames.add(entry.getName());
+ }
+
+ assertThat(zipEntryNames,
+ containsInAnyOrder("directory/file.txt", "empty_directory/", "file.txt"));
+ }
+
+ @Test
+ public void testPackageUploadWithEmptyDirectorySucceeds() throws Exception {
+ Pipe pipe = Pipe.open();
+ File tmpDirectory = tmpFolder.newFolder("folder");
+
+ when(mockGcsUtil.fileSize(any(GcsPath.class)))
+ .thenThrow(new FileNotFoundException("some/path"));
+ when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
+
+ List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
+ ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
+ DataflowPackage target = Iterables.getOnlyElement(targets);
+
+ verify(mockGcsUtil).fileSize(any(GcsPath.class));
+ verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+ verifyNoMoreInteractions(mockGcsUtil);
+
+ assertThat(target.getName(), RegexMatcher.matches("folder-" + HASH_PATTERN + ".jar"));
+ assertThat(target.getLocation(), equalTo(STAGING_PATH + '/' + target.getName()));
+ assertNull(new ZipInputStream(Channels.newInputStream(pipe.source())).getNextEntry());
+ }
+
+ @Test(expected = RuntimeException.class)
+ public void testPackageUploadFailsWhenIOExceptionThrown() throws Exception {
+ File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+ when(mockGcsUtil.fileSize(any(GcsPath.class)))
+ .thenThrow(new FileNotFoundException("some/path"));
+ when(mockGcsUtil.create(any(GcsPath.class), anyString()))
+ .thenThrow(new IOException("Fake Exception: Upload error"));
+
+ try {
+ PackageUtil.stageClasspathElements(
+ ImmutableList.of(tmpFile.getAbsolutePath()),
+ STAGING_PATH, fastNanoClockAndSleeper);
+ } finally {
+ verify(mockGcsUtil).fileSize(any(GcsPath.class));
+ verify(mockGcsUtil, times(5)).create(any(GcsPath.class), anyString());
+ verifyNoMoreInteractions(mockGcsUtil);
+ }
+ }
+
+ @Test
+ public void testPackageUploadFailsWithPermissionsErrorGivesDetailedMessage() throws Exception {
+ File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+ when(mockGcsUtil.fileSize(any(GcsPath.class)))
+ .thenThrow(new FileNotFoundException("some/path"));
+ when(mockGcsUtil.create(any(GcsPath.class), anyString()))
+ .thenThrow(new IOException("Failed to write to GCS path " + STAGING_PATH,
+ googleJsonResponseException(
+ HttpStatusCodes.STATUS_CODE_FORBIDDEN, "Permission denied", "Test message")));
+
+ try {
+ PackageUtil.stageClasspathElements(
+ ImmutableList.of(tmpFile.getAbsolutePath()),
+ STAGING_PATH, fastNanoClockAndSleeper);
+ fail("Expected RuntimeException");
+ } catch (RuntimeException e) {
+ assertTrue("Expected IOException containing detailed message.",
+ e.getCause() instanceof IOException);
+ assertThat(e.getCause().getMessage(),
+ Matchers.allOf(
+ Matchers.containsString("Uploaded failed due to permissions error"),
+ Matchers.containsString(
+ "Stale credentials can be resolved by executing 'gcloud auth login'")));
+ } finally {
+ verify(mockGcsUtil).fileSize(any(GcsPath.class));
+ verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+ verifyNoMoreInteractions(mockGcsUtil);
+ }
+ }
+
+ @Test
+ public void testPackageUploadEventuallySucceeds() throws Exception {
+ Pipe pipe = Pipe.open();
+ File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+ when(mockGcsUtil.fileSize(any(GcsPath.class)))
+ .thenThrow(new FileNotFoundException("some/path"));
+ when(mockGcsUtil.create(any(GcsPath.class), anyString()))
+ .thenThrow(new IOException("Fake Exception: 410 Gone")) // First attempt fails
+ .thenReturn(pipe.sink()); // second attempt succeeds
+
+ try {
+ PackageUtil.stageClasspathElements(
+ ImmutableList.of(tmpFile.getAbsolutePath()),
+ STAGING_PATH,
+ fastNanoClockAndSleeper);
+ } finally {
+ verify(mockGcsUtil).fileSize(any(GcsPath.class));
+ verify(mockGcsUtil, times(2)).create(any(GcsPath.class), anyString());
+ verifyNoMoreInteractions(mockGcsUtil);
+ }
+ }
+
+ @Test
+ public void testPackageUploadIsSkippedWhenFileAlreadyExists() throws Exception {
+ File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+ when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(tmpFile.length());
+
+ PackageUtil.stageClasspathElements(
+ ImmutableList.of(tmpFile.getAbsolutePath()), STAGING_PATH);
+
+ verify(mockGcsUtil).fileSize(any(GcsPath.class));
+ verifyNoMoreInteractions(mockGcsUtil);
+ }
+
+ @Test
+ public void testPackageUploadIsNotSkippedWhenSizesAreDifferent() throws Exception {
+ Pipe pipe = Pipe.open();
+ File tmpDirectory = tmpFolder.newFolder("folder");
+ tmpFolder.newFolder("folder", "empty_directory");
+ tmpFolder.newFolder("folder", "directory");
+ makeFileWithContents("folder/file.txt", "This is a test!");
+ makeFileWithContents("folder/directory/file.txt", "This is also a test!");
+ when(mockGcsUtil.fileSize(any(GcsPath.class))).thenReturn(Long.MAX_VALUE);
+ when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
+
+ PackageUtil.stageClasspathElements(
+ ImmutableList.of(tmpDirectory.getAbsolutePath()), STAGING_PATH);
+
+ verify(mockGcsUtil).fileSize(any(GcsPath.class));
+ verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+ verifyNoMoreInteractions(mockGcsUtil);
+ }
+
+ @Test
+ public void testPackageUploadWithExplicitPackageName() throws Exception {
+ Pipe pipe = Pipe.open();
+ File tmpFile = makeFileWithContents("file.txt", "This is a test!");
+ final String overriddenName = "alias.txt";
+
+ when(mockGcsUtil.fileSize(any(GcsPath.class)))
+ .thenThrow(new FileNotFoundException("some/path"));
+ when(mockGcsUtil.create(any(GcsPath.class), anyString())).thenReturn(pipe.sink());
+
+ List<DataflowPackage> targets = PackageUtil.stageClasspathElements(
+ ImmutableList.of(overriddenName + "=" + tmpFile.getAbsolutePath()), STAGING_PATH);
+ DataflowPackage target = Iterables.getOnlyElement(targets);
+
+ verify(mockGcsUtil).fileSize(any(GcsPath.class));
+ verify(mockGcsUtil).create(any(GcsPath.class), anyString());
+ verifyNoMoreInteractions(mockGcsUtil);
+
+ assertThat(target.getName(), equalTo(overriddenName));
+ assertThat(target.getLocation(),
+ RegexMatcher.matches(STAGING_PATH + "/file-" + HASH_PATTERN + ".txt"));
+ }
+
+ @Test
+ public void testPackageUploadIsSkippedWithNonExistentResource() throws Exception {
+ String nonExistentFile =
+ IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "non-existent-file");
+ assertEquals(Collections.EMPTY_LIST, PackageUtil.stageClasspathElements(
+ ImmutableList.of(nonExistentFile), STAGING_PATH));
+ }
+
+ /**
+ * Builds a fake GoogleJsonResponseException for testing API error handling.
+ */
+ private static GoogleJsonResponseException googleJsonResponseException(
+ final int status, final String reason, final String message) throws IOException {
+ final JsonFactory jsonFactory = new JacksonFactory();
+ HttpTransport transport = new MockHttpTransport() {
+ @Override
+ public LowLevelHttpRequest buildRequest(String method, String url) throws IOException {
+ ErrorInfo errorInfo = new ErrorInfo();
+ errorInfo.setReason(reason);
+ errorInfo.setMessage(message);
+ errorInfo.setFactory(jsonFactory);
+ GenericJson error = new GenericJson();
+ error.set("code", status);
+ error.set("errors", Arrays.asList(errorInfo));
+ error.setFactory(jsonFactory);
+ GenericJson errorResponse = new GenericJson();
+ errorResponse.set("error", error);
+ errorResponse.setFactory(jsonFactory);
+ return new MockLowLevelHttpRequest().setResponse(
+ new MockLowLevelHttpResponse().setContent(errorResponse.toPrettyString())
+ .setContentType(Json.MEDIA_TYPE).setStatusCode(status));
+ }
+ };
+ HttpRequest request =
+ transport.createRequestFactory().buildGetRequest(HttpTesting.SIMPLE_GENERIC_URL);
+ request.setThrowExceptionOnExecuteError(false);
+ HttpResponse response = request.execute();
+ return GoogleJsonResponseException.from(jsonFactory, response);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java
deleted file mode 100644
index 6b9fbb4..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/io/DataflowTextIOTest.java
+++ /dev/null
@@ -1,118 +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 org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.DataflowPipelineRunner;
-import org.apache.beam.sdk.testing.TestDataflowPipelineOptions;
-import org.apache.beam.sdk.util.GcsUtil;
-import org.apache.beam.sdk.util.TestCredential;
-import org.apache.beam.sdk.util.gcsfs.GcsPath;
-
-import com.google.common.collect.ImmutableList;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SeekableByteChannel;
-import java.nio.file.Files;
-import java.nio.file.StandardOpenOption;
-import java.util.List;
-
-/**
- * {@link DataflowPipelineRunner} specific tests for TextIO Read and Write transforms.
- */
-@RunWith(JUnit4.class)
-public class DataflowTextIOTest {
-
- private TestDataflowPipelineOptions buildTestPipelineOptions() {
- TestDataflowPipelineOptions options =
- PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
- options.setGcpCredential(new TestCredential());
- return options;
- }
-
- private GcsUtil buildMockGcsUtil() throws IOException {
- GcsUtil mockGcsUtil = Mockito.mock(GcsUtil.class);
-
- // Any request to open gets a new bogus channel
- Mockito
- .when(mockGcsUtil.open(Mockito.any(GcsPath.class)))
- .then(new Answer<SeekableByteChannel>() {
- @Override
- public SeekableByteChannel answer(InvocationOnMock invocation) throws Throwable {
- return FileChannel.open(
- Files.createTempFile("channel-", ".tmp"),
- StandardOpenOption.CREATE, StandardOpenOption.DELETE_ON_CLOSE);
- }
- });
-
- // Any request for expansion returns a list containing the original GcsPath
- // This is required to pass validation that occurs in TextIO during apply()
- Mockito
- .when(mockGcsUtil.expand(Mockito.any(GcsPath.class)))
- .then(new Answer<List<GcsPath>>() {
- @Override
- public List<GcsPath> answer(InvocationOnMock invocation) throws Throwable {
- return ImmutableList.of((GcsPath) invocation.getArguments()[0]);
- }
- });
-
- return mockGcsUtil;
- }
-
- /**
- * This tests a few corner cases that should not crash.
- */
- @Test
- public void testGoodWildcards() throws Exception {
- TestDataflowPipelineOptions options = buildTestPipelineOptions();
- options.setGcsUtil(buildMockGcsUtil());
-
- Pipeline pipeline = Pipeline.create(options);
-
- applyRead(pipeline, "gs://bucket/foo");
- applyRead(pipeline, "gs://bucket/foo/");
- applyRead(pipeline, "gs://bucket/foo/*");
- applyRead(pipeline, "gs://bucket/foo/?");
- applyRead(pipeline, "gs://bucket/foo/[0-9]");
- applyRead(pipeline, "gs://bucket/foo/*baz*");
- applyRead(pipeline, "gs://bucket/foo/*baz?");
- applyRead(pipeline, "gs://bucket/foo/[0-9]baz?");
- applyRead(pipeline, "gs://bucket/foo/baz/*");
- applyRead(pipeline, "gs://bucket/foo/baz/*wonka*");
- applyRead(pipeline, "gs://bucket/foo/*baz/wonka*");
- applyRead(pipeline, "gs://bucket/foo*/baz");
- applyRead(pipeline, "gs://bucket/foo?/baz");
- applyRead(pipeline, "gs://bucket/foo[0-9]/baz");
-
- // Check that running doesn't fail.
- pipeline.run();
- }
-
- private void applyRead(Pipeline pipeline, String path) {
- pipeline.apply("Read(" + path + ")", TextIO.Read.from(path));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java
deleted file mode 100644
index c3f3a18..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineDebugOptionsTest.java
+++ /dev/null
@@ -1,41 +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.options;
-
-import static org.hamcrest.Matchers.hasEntry;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link DataflowPipelineDebugOptions}. */
-@RunWith(JUnit4.class)
-public class DataflowPipelineDebugOptionsTest {
- @Test
- public void testTransformNameMapping() throws Exception {
- DataflowPipelineDebugOptions options = PipelineOptionsFactory
- .fromArgs(new String[]{"--transformNameMapping={\"a\":\"b\",\"foo\":\"\",\"bar\":\"baz\"}"})
- .as(DataflowPipelineDebugOptions.class);
- assertEquals(3, options.getTransformNameMapping().size());
- assertThat(options.getTransformNameMapping(), hasEntry("a", "b"));
- assertThat(options.getTransformNameMapping(), hasEntry("foo", ""));
- assertThat(options.getTransformNameMapping(), hasEntry("bar", "baz"));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java
deleted file mode 100644
index c9eac56..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowPipelineOptionsTest.java
+++ /dev/null
@@ -1,92 +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.options;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.beam.sdk.testing.ResetDateTimeProvider;
-import org.apache.beam.sdk.testing.RestoreSystemProperties;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link DataflowPipelineOptions}. */
-@RunWith(JUnit4.class)
-public class DataflowPipelineOptionsTest {
- @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
- @Rule public ResetDateTimeProvider resetDateTimeProviderRule = new ResetDateTimeProvider();
-
- @Test
- public void testJobNameIsSet() {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setJobName("TestJobName");
- assertEquals("TestJobName", options.getJobName());
- }
-
- @Test
- public void testUserNameIsNotSet() {
- resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
- System.getProperties().remove("user.name");
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setAppName("TestApplication");
- assertEquals("testapplication--1208190706", options.getJobName());
- assertTrue(options.getJobName().length() <= 40);
- }
-
- @Test
- public void testAppNameAndUserNameAreLong() {
- resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
- System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde");
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setAppName("1234567890123456789012345678901234567890");
- assertEquals(
- "a234567890123456789012345678901234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706",
- options.getJobName());
- }
-
- @Test
- public void testAppNameIsLong() {
- resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
- System.getProperties().put("user.name", "abcde");
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setAppName("1234567890123456789012345678901234567890");
- assertEquals("a234567890123456789012345678901234567890-abcde-1208190706", options.getJobName());
- }
-
- @Test
- public void testUserNameIsLong() {
- resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
- System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde");
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setAppName("1234567890");
- assertEquals("a234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706", options.getJobName());
- }
-
- @Test
- public void testUtf8UserNameAndApplicationNameIsNormalized() {
- resetDateTimeProviderRule.setDateTimeFixed("2014-12-08T19:07:06.698Z");
- System.getProperties().put("user.name", "ði ıntəˈnæʃənəl ");
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setAppName("fəˈnɛtık əsoʊsiˈeıʃn");
- assertEquals("f00n0t0k00so0si0e00n-0i00nt00n000n0l0-1208190706", options.getJobName());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java
deleted file mode 100644
index 18c8085..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowProfilingOptionsTest.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.options;
-
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.hamcrest.Matchers;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/**
- * Tests for {@link DataflowProfilingOptions}.
- */
-@RunWith(JUnit4.class)
-public class DataflowProfilingOptionsTest {
-
- private static final ObjectMapper MAPPER = new ObjectMapper();
-
- @Test
- public void testOptionsObject() throws Exception {
- DataflowPipelineOptions options = PipelineOptionsFactory.fromArgs(new String[] {
- "--enableProfilingAgent", "--profilingAgentConfiguration={\"interval\": 21}"})
- .as(DataflowPipelineOptions.class);
- assertTrue(options.getEnableProfilingAgent());
-
- String json = MAPPER.writeValueAsString(options);
- assertThat(json, Matchers.containsString(
- "\"profilingAgentConfiguration\":{\"interval\":21}"));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.java
deleted file mode 100644
index 47d518d..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/options/DataflowWorkerLoggingOptionsTest.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.sdk.options;
-
-import static org.apache.beam.sdk.options.DataflowWorkerLoggingOptions.Level.WARN;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.beam.sdk.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides;
-
-import com.google.common.collect.ImmutableMap;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link DataflowWorkerLoggingOptions}. */
-@RunWith(JUnit4.class)
-public class DataflowWorkerLoggingOptionsTest {
- private static final ObjectMapper MAPPER = new ObjectMapper();
- @Rule public ExpectedException expectedException = ExpectedException.none();
-
- @Test
- public void testWorkerLogLevelOverrideWithInvalidLogLevel() {
- expectedException.expect(IllegalArgumentException.class);
- expectedException.expectMessage("Unsupported log level");
- WorkerLogLevelOverrides.from(ImmutableMap.of("Name", "FakeLevel"));
- }
-
- @Test
- public void testWorkerLogLevelOverrideForClass() throws Exception {
- assertEquals("{\"org.junit.Test\":\"WARN\"}",
- MAPPER.writeValueAsString(
- new WorkerLogLevelOverrides().addOverrideForClass(Test.class, WARN)));
- }
-
- @Test
- public void testWorkerLogLevelOverrideForPackage() throws Exception {
- assertEquals("{\"org.junit\":\"WARN\"}",
- MAPPER.writeValueAsString(
- new WorkerLogLevelOverrides().addOverrideForPackage(Test.class.getPackage(), WARN)));
- }
-
- @Test
- public void testWorkerLogLevelOverrideForName() throws Exception {
- assertEquals("{\"A\":\"WARN\"}",
- MAPPER.writeValueAsString(
- new WorkerLogLevelOverrides().addOverrideForName("A", WARN)));
- }
-
- @Test
- public void testSerializationAndDeserializationOf() throws Exception {
- String testValue = "{\"A\":\"WARN\"}";
- assertEquals(testValue,
- MAPPER.writeValueAsString(
- MAPPER.readValue(testValue, WorkerLogLevelOverrides.class)));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java
deleted file mode 100644
index 13e120b..0000000
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/sdk/runners/BlockingDataflowPipelineRunnerTest.java
+++ /dev/null
@@ -1,302 +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.runners;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.options.DataflowPipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.ExpectedLogs;
-import org.apache.beam.sdk.testing.TestDataflowPipelineOptions;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.util.MonitoringUtil;
-import org.apache.beam.sdk.util.NoopPathValidator;
-import org.apache.beam.sdk.util.TestCredential;
-
-import org.hamcrest.Description;
-import org.hamcrest.Factory;
-import org.hamcrest.Matcher;
-import org.hamcrest.TypeSafeMatcher;
-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.concurrent.TimeUnit;
-
-/**
- * Tests for BlockingDataflowPipelineRunner.
- */
-@RunWith(JUnit4.class)
-public class BlockingDataflowPipelineRunnerTest {
-
- @Rule
- public ExpectedLogs expectedLogs = ExpectedLogs.none(BlockingDataflowPipelineRunner.class);
-
- @Rule
- public ExpectedException expectedThrown = ExpectedException.none();
-
- /**
- * A {@link Matcher} for a {@link DataflowJobException} that applies an underlying {@link Matcher}
- * to the {@link DataflowPipelineJob} returned by {@link DataflowJobException#getJob()}.
- */
- private static class DataflowJobExceptionMatcher<T extends DataflowJobException>
- extends TypeSafeMatcher<T> {
-
- private final Matcher<DataflowPipelineJob> matcher;
-
- public DataflowJobExceptionMatcher(Matcher<DataflowPipelineJob> matcher) {
- this.matcher = matcher;
- }
-
- @Override
- public boolean matchesSafely(T ex) {
- return matcher.matches(ex.getJob());
- }
-
- @Override
- protected void describeMismatchSafely(T item, Description description) {
- description.appendText("job ");
- matcher.describeMismatch(item.getMessage(), description);
- }
-
- @Override
- public void describeTo(Description description) {
- description.appendText("exception with job matching ");
- description.appendDescriptionOf(matcher);
- }
-
- @Factory
- public static <T extends DataflowJobException> Matcher<T> expectJob(
- Matcher<DataflowPipelineJob> matcher) {
- return new DataflowJobExceptionMatcher<T>(matcher);
- }
- }
-
- /**
- * A {@link Matcher} for a {@link DataflowPipelineJob} that applies an underlying {@link Matcher}
- * to the return value of {@link DataflowPipelineJob#getJobId()}.
- */
- private static class JobIdMatcher<T extends DataflowPipelineJob> extends TypeSafeMatcher<T> {
-
- private final Matcher<String> matcher;
-
- public JobIdMatcher(Matcher<String> matcher) {
- this.matcher = matcher;
- }
-
- @Override
- public boolean matchesSafely(T job) {
- return matcher.matches(job.getJobId());
- }
-
- @Override
- protected void describeMismatchSafely(T item, Description description) {
- description.appendText("jobId ");
- matcher.describeMismatch(item.getJobId(), description);
- }
-
- @Override
- public void describeTo(Description description) {
- description.appendText("job with jobId ");
- description.appendDescriptionOf(matcher);
- }
-
- @Factory
- public static <T extends DataflowPipelineJob> Matcher<T> expectJobId(final String jobId) {
- return new JobIdMatcher<T>(equalTo(jobId));
- }
- }
-
- /**
- * A {@link Matcher} for a {@link DataflowJobUpdatedException} that applies an underlying
- * {@link Matcher} to the {@link DataflowPipelineJob} returned by
- * {@link DataflowJobUpdatedException#getReplacedByJob()}.
- */
- private static class ReplacedByJobMatcher<T extends DataflowJobUpdatedException>
- extends TypeSafeMatcher<T> {
-
- private final Matcher<DataflowPipelineJob> matcher;
-
- public ReplacedByJobMatcher(Matcher<DataflowPipelineJob> matcher) {
- this.matcher = matcher;
- }
-
- @Override
- public boolean matchesSafely(T ex) {
- return matcher.matches(ex.getReplacedByJob());
- }
-
- @Override
- protected void describeMismatchSafely(T item, Description description) {
- description.appendText("job ");
- matcher.describeMismatch(item.getMessage(), description);
- }
-
- @Override
- public void describeTo(Description description) {
- description.appendText("exception with replacedByJob() ");
- description.appendDescriptionOf(matcher);
- }
-
- @Factory
- public static <T extends DataflowJobUpdatedException> Matcher<T> expectReplacedBy(
- Matcher<DataflowPipelineJob> matcher) {
- return new ReplacedByJobMatcher<T>(matcher);
- }
- }
-
- /**
- * Creates a mocked {@link DataflowPipelineJob} with the given {@code projectId} and {@code jobId}
- * that will immediately terminate in the provided {@code terminalState}.
- *
- * <p>The return value may be further mocked.
- */
- private DataflowPipelineJob createMockJob(
- String projectId, String jobId, State terminalState) throws Exception {
- DataflowPipelineJob mockJob = mock(DataflowPipelineJob.class);
- when(mockJob.getProjectId()).thenReturn(projectId);
- when(mockJob.getJobId()).thenReturn(jobId);
- when(mockJob.waitToFinish(
- anyLong(), isA(TimeUnit.class), isA(MonitoringUtil.JobMessagesHandler.class)))
- .thenReturn(terminalState);
- return mockJob;
- }
-
- /**
- * Returns a {@link BlockingDataflowPipelineRunner} that will return the provided a job to return.
- * Some {@link PipelineOptions} will be extracted from the job, such as the project ID.
- */
- private BlockingDataflowPipelineRunner createMockRunner(DataflowPipelineJob job)
- throws Exception {
- DataflowPipelineRunner mockRunner = mock(DataflowPipelineRunner.class);
- TestDataflowPipelineOptions options =
- PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
- options.setProject(job.getProjectId());
-
- when(mockRunner.run(isA(Pipeline.class))).thenReturn(job);
-
- return new BlockingDataflowPipelineRunner(mockRunner, options);
- }
-
- /**
- * Tests that the {@link BlockingDataflowPipelineRunner} returns normally when a job terminates in
- * the {@link State#DONE DONE} state.
- */
- @Test
- public void testJobDoneComplete() throws Exception {
- createMockRunner(createMockJob("testJobDone-projectId", "testJobDone-jobId", State.DONE))
- .run(TestPipeline.create());
- expectedLogs.verifyInfo("Job finished with status DONE");
- }
-
- /**
- * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
- * when a job terminates in the {@link State#FAILED FAILED} state.
- */
- @Test
- public void testFailedJobThrowsException() throws Exception {
- expectedThrown.expect(DataflowJobExecutionException.class);
- expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
- JobIdMatcher.expectJobId("testFailedJob-jobId")));
- createMockRunner(createMockJob("testFailedJob-projectId", "testFailedJob-jobId", State.FAILED))
- .run(TestPipeline.create());
- }
-
- /**
- * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
- * when a job terminates in the {@link State#CANCELLED CANCELLED} state.
- */
- @Test
- public void testCancelledJobThrowsException() throws Exception {
- expectedThrown.expect(DataflowJobCancelledException.class);
- expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
- JobIdMatcher.expectJobId("testCancelledJob-jobId")));
- createMockRunner(
- createMockJob("testCancelledJob-projectId", "testCancelledJob-jobId", State.CANCELLED))
- .run(TestPipeline.create());
- }
-
- /**
- * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
- * when a job terminates in the {@link State#UPDATED UPDATED} state.
- */
- @Test
- public void testUpdatedJobThrowsException() throws Exception {
- expectedThrown.expect(DataflowJobUpdatedException.class);
- expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
- JobIdMatcher.expectJobId("testUpdatedJob-jobId")));
- expectedThrown.expect(ReplacedByJobMatcher.expectReplacedBy(
- JobIdMatcher.expectJobId("testUpdatedJob-replacedByJobId")));
- DataflowPipelineJob job =
- createMockJob("testUpdatedJob-projectId", "testUpdatedJob-jobId", State.UPDATED);
- DataflowPipelineJob replacedByJob =
- createMockJob("testUpdatedJob-projectId", "testUpdatedJob-replacedByJobId", State.DONE);
- when(job.getReplacedByJob()).thenReturn(replacedByJob);
- createMockRunner(job).run(TestPipeline.create());
- }
-
- /**
- * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
- * when a job terminates in the {@link State#UNKNOWN UNKNOWN} state, indicating that the
- * Dataflow service returned a state that the SDK is unfamiliar with (possibly because it
- * is an old SDK relative the service).
- */
- @Test
- public void testUnknownJobThrowsException() throws Exception {
- expectedThrown.expect(IllegalStateException.class);
- createMockRunner(
- createMockJob("testUnknownJob-projectId", "testUnknownJob-jobId", State.UNKNOWN))
- .run(TestPipeline.create());
- }
-
- /**
- * Tests that the {@link BlockingDataflowPipelineRunner} throws the appropriate exception
- * when a job returns a {@code null} state, indicating that it failed to contact the service,
- * including all of its built-in resilience logic.
- */
- @Test
- public void testNullJobThrowsException() throws Exception {
- expectedThrown.expect(DataflowServiceException.class);
- expectedThrown.expect(DataflowJobExceptionMatcher.expectJob(
- JobIdMatcher.expectJobId("testNullJob-jobId")));
- createMockRunner(createMockJob("testNullJob-projectId", "testNullJob-jobId", null))
- .run(TestPipeline.create());
- }
-
- @Test
- public void testToString() {
- DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
- options.setJobName("TestJobName");
- options.setProject("test-project");
- options.setTempLocation("gs://test/temp/location");
- options.setGcpCredential(new TestCredential());
- options.setPathValidatorClass(NoopPathValidator.class);
- assertEquals("BlockingDataflowPipelineRunner#testjobname",
- BlockingDataflowPipelineRunner.fromOptions(options).toString());
- }
-}
[16/21] incubator-beam git commit: Reorganize Java packages in the
sources of the Google Cloud Dataflow runner
Posted by dh...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/02190985/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java
new file mode 100644
index 0000000..0fc095a
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRunner.java
@@ -0,0 +1,3025 @@
+/*
+ * 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.dataflow;
+
+import static org.apache.beam.sdk.util.StringUtils.approximatePTransformName;
+import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName;
+import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification;
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTranslator;
+import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext;
+import org.apache.beam.runners.dataflow.internal.AssignWindows;
+import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms;
+import org.apache.beam.runners.dataflow.internal.PubsubIOTranslator;
+import org.apache.beam.runners.dataflow.internal.ReadTranslator;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions;
+import org.apache.beam.runners.dataflow.util.DataflowTransport;
+import org.apache.beam.runners.dataflow.util.MonitoringUtil;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.PipelineResult.State;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.CannotProvideCoderException;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.coders.IterableCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.FileBasedSink;
+import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.ShardNameTemplate;
+import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.Write;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.options.StreamingOptions;
+import org.apache.beam.sdk.runners.AggregatorPipelineExtractor;
+import org.apache.beam.sdk.runners.PipelineRunner;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.runners.worker.IsmFormat;
+import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecord;
+import org.apache.beam.sdk.runners.worker.IsmFormat.IsmRecordCoder;
+import org.apache.beam.sdk.runners.worker.IsmFormat.MetadataKeyCoder;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.Create;
+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.SerializableFunction;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.AfterPane;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.apache.beam.sdk.util.InstanceBuilder;
+import org.apache.beam.sdk.util.PCollectionViews;
+import org.apache.beam.sdk.util.PathValidator;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.util.ReleaseInfo;
+import org.apache.beam.sdk.util.Reshuffle;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.ValueWithRecordId;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
+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.PCollection.IsBounded;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
+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.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+
+import com.google.api.client.googleapis.json.GoogleJsonResponseException;
+import com.google.api.services.clouddebugger.v2.Clouddebugger;
+import com.google.api.services.clouddebugger.v2.model.Debuggee;
+import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest;
+import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse;
+import com.google.api.services.dataflow.Dataflow;
+import com.google.api.services.dataflow.model.DataflowPackage;
+import com.google.api.services.dataflow.model.Job;
+import com.google.api.services.dataflow.model.ListJobsResponse;
+import com.google.api.services.dataflow.model.WorkerPool;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.base.Utf8;
+import com.google.common.collect.ForwardingMap;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.joda.time.DateTimeUtils;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Duration;
+import org.joda.time.format.DateTimeFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+/**
+ * A {@link PipelineRunner} that executes the operations in the
+ * pipeline by first translating them to the Dataflow representation
+ * using the {@link DataflowPipelineTranslator} and then submitting
+ * them to a Dataflow service for execution.
+ *
+ * <p><h3>Permissions</h3>
+ * When reading from a Dataflow source or writing to a Dataflow sink using
+ * {@code DataflowPipelineRunner}, the Google cloudservices account and the Google compute engine
+ * service account of the GCP project running the Dataflow Job will need access to the corresponding
+ * source/sink.
+ *
+ * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
+ * Dataflow Security and Permissions</a> for more details.
+ */
+public class DataflowPipelineRunner extends PipelineRunner<DataflowPipelineJob> {
+ private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineRunner.class);
+
+ /** Provided configuration options. */
+ private final DataflowPipelineOptions options;
+
+ /** Client for the Dataflow service. This is used to actually submit jobs. */
+ private final Dataflow dataflowClient;
+
+ /** Translator for this DataflowPipelineRunner, based on options. */
+ private final DataflowPipelineTranslator translator;
+
+ /** Custom transforms implementations. */
+ private final Map<Class<?>, Class<?>> overrides;
+
+ /** A set of user defined functions to invoke at different points in execution. */
+ private DataflowPipelineRunnerHooks hooks;
+
+ // Environment version information.
+ private static final String ENVIRONMENT_MAJOR_VERSION = "4";
+
+ // Default Docker container images that execute Dataflow worker harness, residing in Google
+ // Container Registry, separately for Batch and Streaming.
+ public static final String BATCH_WORKER_HARNESS_CONTAINER_IMAGE
+ = "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160422";
+ public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE
+ = "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160422";
+
+ // The limit of CreateJob request size.
+ private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024;
+
+ private final Set<PCollection<?>> pcollectionsRequiringIndexedFormat;
+
+ /**
+ * Project IDs must contain lowercase letters, digits, or dashes.
+ * IDs must start with a letter and may not end with a dash.
+ * This regex isn't exact - this allows for patterns that would be rejected by
+ * the service, but this is sufficient for basic validation of project IDs.
+ */
+ public static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]+[a-z0-9]";
+
+ /**
+ * Construct a runner from the provided options.
+ *
+ * @param options Properties that configure the runner.
+ * @return The newly created runner.
+ */
+ public static DataflowPipelineRunner fromOptions(PipelineOptions options) {
+ // (Re-)register standard IO factories. Clobbers any prior credentials.
+ IOChannelUtils.registerStandardIOFactories(options);
+
+ DataflowPipelineOptions dataflowOptions =
+ PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options);
+ ArrayList<String> missing = new ArrayList<>();
+
+ if (dataflowOptions.getAppName() == null) {
+ missing.add("appName");
+ }
+ if (missing.size() > 0) {
+ throw new IllegalArgumentException(
+ "Missing required values: " + Joiner.on(',').join(missing));
+ }
+
+ PathValidator validator = dataflowOptions.getPathValidator();
+ Preconditions.checkArgument(!(Strings.isNullOrEmpty(dataflowOptions.getTempLocation())
+ && Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())),
+ "Missing required value: at least one of tempLocation or stagingLocation must be set.");
+
+ if (dataflowOptions.getStagingLocation() != null) {
+ validator.validateOutputFilePrefixSupported(dataflowOptions.getStagingLocation());
+ }
+ if (dataflowOptions.getTempLocation() != null) {
+ validator.validateOutputFilePrefixSupported(dataflowOptions.getTempLocation());
+ }
+ if (Strings.isNullOrEmpty(dataflowOptions.getTempLocation())) {
+ dataflowOptions.setTempLocation(dataflowOptions.getStagingLocation());
+ } else if (Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())) {
+ try {
+ dataflowOptions.setStagingLocation(
+ IOChannelUtils.resolve(dataflowOptions.getTempLocation(), "staging"));
+ } catch (IOException e) {
+ throw new IllegalArgumentException("Unable to resolve PipelineOptions.stagingLocation "
+ + "from PipelineOptions.tempLocation. Please set the staging location explicitly.", e);
+ }
+ }
+
+ if (dataflowOptions.getFilesToStage() == null) {
+ dataflowOptions.setFilesToStage(detectClassPathResourcesToStage(
+ DataflowPipelineRunner.class.getClassLoader()));
+ LOG.info("PipelineOptions.filesToStage was not specified. "
+ + "Defaulting to files from the classpath: will stage {} files. "
+ + "Enable logging at DEBUG level to see which files will be staged.",
+ dataflowOptions.getFilesToStage().size());
+ LOG.debug("Classpath elements: {}", dataflowOptions.getFilesToStage());
+ }
+
+ // Verify jobName according to service requirements, truncating converting to lowercase if
+ // necessary.
+ String jobName =
+ dataflowOptions
+ .getJobName()
+ .toLowerCase();
+ checkArgument(
+ jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"),
+ "JobName invalid; the name must consist of only the characters "
+ + "[-a-z0-9], starting with a letter and ending with a letter "
+ + "or number");
+ if (!jobName.equals(dataflowOptions.getJobName())) {
+ LOG.info(
+ "PipelineOptions.jobName did not match the service requirements. "
+ + "Using {} instead of {}.",
+ jobName,
+ dataflowOptions.getJobName());
+ }
+ dataflowOptions.setJobName(jobName);
+
+ // Verify project
+ String project = dataflowOptions.getProject();
+ if (project.matches("[0-9]*")) {
+ throw new IllegalArgumentException("Project ID '" + project
+ + "' invalid. Please make sure you specified the Project ID, not project number.");
+ } else if (!project.matches(PROJECT_ID_REGEXP)) {
+ throw new IllegalArgumentException("Project ID '" + project
+ + "' invalid. Please make sure you specified the Project ID, not project description.");
+ }
+
+ DataflowPipelineDebugOptions debugOptions =
+ dataflowOptions.as(DataflowPipelineDebugOptions.class);
+ // Verify the number of worker threads is a valid value
+ if (debugOptions.getNumberOfWorkerHarnessThreads() < 0) {
+ throw new IllegalArgumentException("Number of worker harness threads '"
+ + debugOptions.getNumberOfWorkerHarnessThreads()
+ + "' invalid. Please make sure the value is non-negative.");
+ }
+
+ return new DataflowPipelineRunner(dataflowOptions);
+ }
+
+ @VisibleForTesting protected DataflowPipelineRunner(DataflowPipelineOptions options) {
+ this.options = options;
+ this.dataflowClient = options.getDataflowClient();
+ this.translator = DataflowPipelineTranslator.fromOptions(options);
+ this.pcollectionsRequiringIndexedFormat = new HashSet<>();
+ this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>();
+
+ if (options.isStreaming()) {
+ overrides = ImmutableMap.<Class<?>, Class<?>>builder()
+ .put(Combine.GloballyAsSingletonView.class, StreamingCombineGloballyAsSingletonView.class)
+ .put(Create.Values.class, StreamingCreate.class)
+ .put(View.AsMap.class, StreamingViewAsMap.class)
+ .put(View.AsMultimap.class, StreamingViewAsMultimap.class)
+ .put(View.AsSingleton.class, StreamingViewAsSingleton.class)
+ .put(View.AsList.class, StreamingViewAsList.class)
+ .put(View.AsIterable.class, StreamingViewAsIterable.class)
+ .put(Write.Bound.class, StreamingWrite.class)
+ .put(PubsubIO.Write.Bound.class, StreamingPubsubIOWrite.class)
+ .put(Read.Unbounded.class, StreamingUnboundedRead.class)
+ .put(Read.Bounded.class, UnsupportedIO.class)
+ .put(AvroIO.Read.Bound.class, UnsupportedIO.class)
+ .put(AvroIO.Write.Bound.class, UnsupportedIO.class)
+ .put(BigQueryIO.Read.Bound.class, UnsupportedIO.class)
+ .put(TextIO.Read.Bound.class, UnsupportedIO.class)
+ .put(TextIO.Write.Bound.class, UnsupportedIO.class)
+ .put(Window.Bound.class, AssignWindows.class)
+ .build();
+ } else {
+ ImmutableMap.Builder<Class<?>, Class<?>> builder = ImmutableMap.<Class<?>, Class<?>>builder();
+ builder.put(Read.Unbounded.class, UnsupportedIO.class);
+ builder.put(Window.Bound.class, AssignWindows.class);
+ builder.put(Write.Bound.class, BatchWrite.class);
+ builder.put(AvroIO.Write.Bound.class, BatchAvroIOWrite.class);
+ builder.put(TextIO.Write.Bound.class, BatchTextIOWrite.class);
+ if (options.getExperiments() == null
+ || !options.getExperiments().contains("disable_ism_side_input")) {
+ builder.put(View.AsMap.class, BatchViewAsMap.class);
+ builder.put(View.AsMultimap.class, BatchViewAsMultimap.class);
+ builder.put(View.AsSingleton.class, BatchViewAsSingleton.class);
+ builder.put(View.AsList.class, BatchViewAsList.class);
+ builder.put(View.AsIterable.class, BatchViewAsIterable.class);
+ }
+ overrides = builder.build();
+ }
+ }
+
+ /**
+ * Applies the given transform to the input. For transforms with customized definitions
+ * for the Dataflow pipeline runner, the application is intercepted and modified here.
+ */
+ @Override
+ public <OutputT extends POutput, InputT extends PInput> OutputT apply(
+ PTransform<InputT, OutputT> transform, InputT input) {
+
+ if (Combine.GroupedValues.class.equals(transform.getClass())
+ || GroupByKey.class.equals(transform.getClass())) {
+
+ // For both Dataflow runners (streaming and batch), GroupByKey and GroupedValues are
+ // primitives. Returning a primitive output instead of the expanded definition
+ // signals to the translator that translation is necessary.
+ @SuppressWarnings("unchecked")
+ PCollection<?> pc = (PCollection<?>) input;
+ @SuppressWarnings("unchecked")
+ OutputT outputT = (OutputT) PCollection.createPrimitiveOutputInternal(
+ pc.getPipeline(),
+ transform instanceof GroupByKey
+ ? ((GroupByKey<?, ?>) transform).updateWindowingStrategy(pc.getWindowingStrategy())
+ : pc.getWindowingStrategy(),
+ pc.isBounded());
+ return outputT;
+ } else if (Window.Bound.class.equals(transform.getClass())) {
+ /*
+ * TODO: make this the generic way overrides are applied (using super.apply() rather than
+ * Pipeline.applyTransform(); this allows the apply method to be replaced without inserting
+ * additional nodes into the graph.
+ */
+ // casting to wildcard
+ @SuppressWarnings("unchecked")
+ OutputT windowed = (OutputT) applyWindow((Window.Bound<?>) transform, (PCollection<?>) input);
+ return windowed;
+ } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass())
+ && ((PCollectionList<?>) input).size() == 0) {
+ return (OutputT) Pipeline.applyTransform(input, Create.of());
+ } else if (overrides.containsKey(transform.getClass())) {
+ // It is the responsibility of whoever constructs overrides to ensure this is type safe.
+ @SuppressWarnings("unchecked")
+ Class<PTransform<InputT, OutputT>> transformClass =
+ (Class<PTransform<InputT, OutputT>>) transform.getClass();
+
+ @SuppressWarnings("unchecked")
+ Class<PTransform<InputT, OutputT>> customTransformClass =
+ (Class<PTransform<InputT, OutputT>>) overrides.get(transform.getClass());
+
+ PTransform<InputT, OutputT> customTransform =
+ InstanceBuilder.ofType(customTransformClass)
+ .withArg(DataflowPipelineRunner.class, this)
+ .withArg(transformClass, transform)
+ .build();
+
+ return Pipeline.applyTransform(input, customTransform);
+ } else {
+ return super.apply(transform, input);
+ }
+ }
+
+ private <T> PCollection<T> applyWindow(
+ Window.Bound<?> intitialTransform, PCollection<?> initialInput) {
+ // types are matched at compile time
+ @SuppressWarnings("unchecked")
+ Window.Bound<T> transform = (Window.Bound<T>) intitialTransform;
+ @SuppressWarnings("unchecked")
+ PCollection<T> input = (PCollection<T>) initialInput;
+ return super.apply(new AssignWindows<>(transform), input);
+ }
+
+ private String debuggerMessage(String projectId, String uniquifier) {
+ return String.format("To debug your job, visit Google Cloud Debugger at: "
+ + "https://console.developers.google.com/debug?project=%s&dbgee=%s",
+ projectId, uniquifier);
+ }
+
+ private void maybeRegisterDebuggee(DataflowPipelineOptions options, String uniquifier) {
+ if (!options.getEnableCloudDebugger()) {
+ return;
+ }
+
+ if (options.getDebuggee() != null) {
+ throw new RuntimeException("Should not specify the debuggee");
+ }
+
+ Clouddebugger debuggerClient = DataflowTransport.newClouddebuggerClient(options).build();
+ Debuggee debuggee = registerDebuggee(debuggerClient, uniquifier);
+ options.setDebuggee(debuggee);
+
+ System.out.println(debuggerMessage(options.getProject(), debuggee.getUniquifier()));
+ }
+
+ private Debuggee registerDebuggee(Clouddebugger debuggerClient, String uniquifier) {
+ RegisterDebuggeeRequest registerReq = new RegisterDebuggeeRequest();
+ registerReq.setDebuggee(new Debuggee()
+ .setProject(options.getProject())
+ .setUniquifier(uniquifier)
+ .setDescription(uniquifier)
+ .setAgentVersion("google.com/cloud-dataflow-java/v1"));
+
+ try {
+ RegisterDebuggeeResponse registerResponse =
+ debuggerClient.controller().debuggees().register(registerReq).execute();
+ Debuggee debuggee = registerResponse.getDebuggee();
+ if (debuggee.getStatus() != null && debuggee.getStatus().getIsError()) {
+ throw new RuntimeException("Unable to register with the debugger: " +
+ debuggee.getStatus().getDescription().getFormat());
+ }
+
+ return debuggee;
+ } catch (IOException e) {
+ throw new RuntimeException("Unable to register with the debugger: ", e);
+ }
+ }
+
+ @Override
+ public DataflowPipelineJob run(Pipeline pipeline) {
+ logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline);
+
+ LOG.info("Executing pipeline on the Dataflow Service, which will have billing implications "
+ + "related to Google Compute Engine usage and other Google Cloud Services.");
+
+ List<DataflowPackage> packages = options.getStager().stageFiles();
+
+
+ // Set a unique client_request_id in the CreateJob request.
+ // This is used to ensure idempotence of job creation across retried
+ // attempts to create a job. Specifically, if the service returns a job with
+ // a different client_request_id, it means the returned one is a different
+ // job previously created with the same job name, and that the job creation
+ // has been effectively rejected. The SDK should return
+ // Error::Already_Exists to user in that case.
+ int randomNum = new Random().nextInt(9000) + 1000;
+ String requestId = DateTimeFormat.forPattern("YYYYMMddHHmmssmmm").withZone(DateTimeZone.UTC)
+ .print(DateTimeUtils.currentTimeMillis()) + "_" + randomNum;
+
+ // Try to create a debuggee ID. This must happen before the job is translated since it may
+ // update the options.
+ DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
+ maybeRegisterDebuggee(dataflowOptions, requestId);
+
+ JobSpecification jobSpecification =
+ translator.translate(pipeline, this, packages);
+ Job newJob = jobSpecification.getJob();
+ newJob.setClientRequestId(requestId);
+
+ String version = ReleaseInfo.getReleaseInfo().getVersion();
+ System.out.println("Dataflow SDK version: " + version);
+
+ newJob.getEnvironment().setUserAgent(ReleaseInfo.getReleaseInfo());
+ // The Dataflow Service may write to the temporary directory directly, so
+ // must be verified.
+ if (!Strings.isNullOrEmpty(options.getTempLocation())) {
+ newJob.getEnvironment().setTempStoragePrefix(
+ dataflowOptions.getPathValidator().verifyPath(options.getTempLocation()));
+ }
+ newJob.getEnvironment().setDataset(options.getTempDatasetId());
+ newJob.getEnvironment().setExperiments(options.getExperiments());
+
+ // Set the Docker container image that executes Dataflow worker harness, residing in Google
+ // Container Registry. Translator is guaranteed to create a worker pool prior to this point.
+ String workerHarnessContainerImage =
+ options.as(DataflowPipelineWorkerPoolOptions.class)
+ .getWorkerHarnessContainerImage();
+ for (WorkerPool workerPool : newJob.getEnvironment().getWorkerPools()) {
+ workerPool.setWorkerHarnessContainerImage(workerHarnessContainerImage);
+ }
+
+ // Requirements about the service.
+ Map<String, Object> environmentVersion = new HashMap<>();
+ environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_MAJOR_KEY, ENVIRONMENT_MAJOR_VERSION);
+ newJob.getEnvironment().setVersion(environmentVersion);
+ // Default jobType is JAVA_BATCH_AUTOSCALING: A Java job with workers that the job can
+ // autoscale if specified.
+ String jobType = "JAVA_BATCH_AUTOSCALING";
+
+ if (options.isStreaming()) {
+ jobType = "STREAMING";
+ }
+ environmentVersion.put(PropertyNames.ENVIRONMENT_VERSION_JOB_TYPE_KEY, jobType);
+
+ if (hooks != null) {
+ hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment());
+ }
+
+ if (!Strings.isNullOrEmpty(options.getDataflowJobFile())) {
+ try (PrintWriter printWriter = new PrintWriter(
+ new File(options.getDataflowJobFile()))) {
+ String workSpecJson = DataflowPipelineTranslator.jobToString(newJob);
+ printWriter.print(workSpecJson);
+ LOG.info("Printed workflow specification to {}", options.getDataflowJobFile());
+ } catch (IllegalStateException ex) {
+ LOG.warn("Cannot translate workflow spec to json for debug.");
+ } catch (FileNotFoundException ex) {
+ LOG.warn("Cannot create workflow spec output file.");
+ }
+ }
+
+ String jobIdToUpdate = null;
+ if (options.isUpdate()) {
+ jobIdToUpdate = getJobIdFromName(options.getJobName());
+ newJob.setTransformNameMapping(options.getTransformNameMapping());
+ newJob.setReplaceJobId(jobIdToUpdate);
+ }
+ Job jobResult;
+ try {
+ jobResult = dataflowClient
+ .projects()
+ .jobs()
+ .create(options.getProject(), newJob)
+ .execute();
+ } catch (GoogleJsonResponseException e) {
+ String errorMessages = "Unexpected errors";
+ if (e.getDetails() != null) {
+ if (Utf8.encodedLength(newJob.toString()) >= CREATE_JOB_REQUEST_LIMIT_BYTES) {
+ errorMessages = "The size of the serialized JSON representation of the pipeline "
+ + "exceeds the allowable limit. "
+ + "For more information, please check the FAQ link below:\n"
+ + "https://cloud.google.com/dataflow/faq";
+ } else {
+ errorMessages = e.getDetails().getMessage();
+ }
+ }
+ throw new RuntimeException("Failed to create a workflow job: " + errorMessages, e);
+ } catch (IOException e) {
+ throw new RuntimeException("Failed to create a workflow job", e);
+ }
+
+ // Obtain all of the extractors from the PTransforms used in the pipeline so the
+ // DataflowPipelineJob has access to them.
+ AggregatorPipelineExtractor aggregatorExtractor = new AggregatorPipelineExtractor(pipeline);
+ Map<Aggregator<?, ?>, Collection<PTransform<?, ?>>> aggregatorSteps =
+ aggregatorExtractor.getAggregatorSteps();
+
+ DataflowAggregatorTransforms aggregatorTransforms =
+ new DataflowAggregatorTransforms(aggregatorSteps, jobSpecification.getStepNames());
+
+ // Use a raw client for post-launch monitoring, as status calls may fail
+ // regularly and need not be retried automatically.
+ DataflowPipelineJob dataflowPipelineJob =
+ new DataflowPipelineJob(options.getProject(), jobResult.getId(),
+ DataflowTransport.newRawDataflowClient(options).build(), aggregatorTransforms);
+
+ // If the service returned client request id, the SDK needs to compare it
+ // with the original id generated in the request, if they are not the same
+ // (i.e., the returned job is not created by this request), throw
+ // DataflowJobAlreadyExistsException or DataflowJobAlreadyUpdatedExcetpion
+ // depending on whether this is a reload or not.
+ if (jobResult.getClientRequestId() != null && !jobResult.getClientRequestId().isEmpty()
+ && !jobResult.getClientRequestId().equals(requestId)) {
+ // If updating a job.
+ if (options.isUpdate()) {
+ throw new DataflowJobAlreadyUpdatedException(dataflowPipelineJob,
+ String.format("The job named %s with id: %s has already been updated into job id: %s "
+ + "and cannot be updated again.",
+ newJob.getName(), jobIdToUpdate, jobResult.getId()));
+ } else {
+ throw new DataflowJobAlreadyExistsException(dataflowPipelineJob,
+ String.format("There is already an active job named %s with id: %s. If you want "
+ + "to submit a second job, try again by setting a different name using --jobName.",
+ newJob.getName(), jobResult.getId()));
+ }
+ }
+
+ LOG.info("To access the Dataflow monitoring console, please navigate to {}",
+ MonitoringUtil.getJobMonitoringPageURL(options.getProject(), jobResult.getId()));
+ System.out.println("Submitted job: " + jobResult.getId());
+
+ LOG.info("To cancel the job using the 'gcloud' tool, run:\n> {}",
+ MonitoringUtil.getGcloudCancelCommand(options, jobResult.getId()));
+
+ return dataflowPipelineJob;
+ }
+
+ /**
+ * Returns the DataflowPipelineTranslator associated with this object.
+ */
+ public DataflowPipelineTranslator getTranslator() {
+ return translator;
+ }
+
+ /**
+ * Sets callbacks to invoke during execution see {@code DataflowPipelineRunnerHooks}.
+ */
+ @Experimental
+ public void setHooks(DataflowPipelineRunnerHooks hooks) {
+ this.hooks = hooks;
+ }
+
+ /////////////////////////////////////////////////////////////////////////////
+
+ /** Outputs a warning about PCollection views without deterministic key coders. */
+ private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) {
+ // We need to wait till this point to determine the names of the transforms since only
+ // at this time do we know the hierarchy of the transforms otherwise we could
+ // have just recorded the full names during apply time.
+ if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) {
+ final SortedSet<String> ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>();
+ pipeline.traverseTopologically(new PipelineVisitor() {
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {
+ if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
+ ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
+ }
+ }
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) {
+ ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName());
+ }
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ }
+ });
+
+ LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} "
+ + "because the key coder is not deterministic. Falling back to singleton implementation "
+ + "which may cause memory and/or performance problems. Future major versions of "
+ + "Dataflow will require deterministic key coders.",
+ ptransformViewNamesWithNonDeterministicKeyCoders);
+ }
+ }
+
+ /**
+ * Returns true if the passed in {@link PCollection} needs to be materialiazed using
+ * an indexed format.
+ */
+ boolean doesPCollectionRequireIndexedFormat(PCollection<?> pcol) {
+ return pcollectionsRequiringIndexedFormat.contains(pcol);
+ }
+
+ /**
+ * Marks the passed in {@link PCollection} as requiring to be materialized using
+ * an indexed format.
+ */
+ private void addPCollectionRequiringIndexedFormat(PCollection<?> pcol) {
+ pcollectionsRequiringIndexedFormat.add(pcol);
+ }
+
+ /** A set of {@link View}s with non-deterministic key coders. */
+ Set<PTransform<?, ?>> ptransformViewsWithNonDeterministicKeyCoders;
+
+ /**
+ * Records that the {@link PTransform} requires a deterministic key coder.
+ */
+ private void recordViewUsesNonDeterministicKeyCoder(PTransform<?, ?> ptransform) {
+ ptransformViewsWithNonDeterministicKeyCoders.add(ptransform);
+ }
+
+ /**
+ * A {@link GroupByKey} transform for the {@link DataflowPipelineRunner} which sorts
+ * values using the secondary key {@code K2}.
+ *
+ * <p>The {@link PCollection} created created by this {@link PTransform} will have values in
+ * the empty window. Care must be taken *afterwards* to either re-window
+ * (using {@link Window#into}) or only use {@link PTransform}s that do not depend on the
+ * values being within a window.
+ */
+ static class GroupByKeyAndSortValuesOnly<K1, K2, V>
+ extends PTransform<PCollection<KV<K1, KV<K2, V>>>, PCollection<KV<K1, Iterable<KV<K2, V>>>>> {
+ private GroupByKeyAndSortValuesOnly() {
+ }
+
+ @Override
+ public PCollection<KV<K1, Iterable<KV<K2, V>>>> apply(PCollection<KV<K1, KV<K2, V>>> input) {
+ PCollection<KV<K1, Iterable<KV<K2, V>>>> rval =
+ PCollection.<KV<K1, Iterable<KV<K2, V>>>>createPrimitiveOutputInternal(
+ input.getPipeline(),
+ WindowingStrategy.globalDefault(),
+ IsBounded.BOUNDED);
+
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ KvCoder<K1, KV<K2, V>> inputCoder = (KvCoder) input.getCoder();
+ rval.setCoder(
+ KvCoder.of(inputCoder.getKeyCoder(),
+ IterableCoder.of(inputCoder.getValueCoder())));
+ return rval;
+ }
+ }
+
+ /**
+ * A {@link PTransform} that groups the values by a hash of the window's byte representation
+ * and sorts the values using the windows byte representation.
+ */
+ private static class GroupByWindowHashAsKeyAndWindowAsSortKey<T, W extends BoundedWindow> extends
+ PTransform<PCollection<T>, PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>>> {
+
+ /**
+ * A {@link DoFn} that for each element outputs a {@code KV} structure suitable for
+ * grouping by the hash of the window's byte representation and sorting the grouped values
+ * using the window's byte representation.
+ */
+ @SystemDoFnInternal
+ private static class UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W extends BoundedWindow>
+ extends DoFn<T, KV<Integer, KV<W, WindowedValue<T>>>> implements DoFn.RequiresWindowAccess {
+
+ private final IsmRecordCoder<?> ismCoderForHash;
+ private UseWindowHashAsKeyAndWindowAsSortKeyDoFn(IsmRecordCoder<?> ismCoderForHash) {
+ this.ismCoderForHash = ismCoderForHash;
+ }
+
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ @SuppressWarnings("unchecked")
+ W window = (W) c.window();
+ c.output(
+ KV.of(ismCoderForHash.hash(ImmutableList.of(window)),
+ KV.of(window,
+ WindowedValue.of(
+ c.element(),
+ c.timestamp(),
+ c.window(),
+ c.pane()))));
+ }
+ }
+
+ private final IsmRecordCoder<?> ismCoderForHash;
+ private GroupByWindowHashAsKeyAndWindowAsSortKey(IsmRecordCoder<?> ismCoderForHash) {
+ this.ismCoderForHash = ismCoderForHash;
+ }
+
+ @Override
+ public PCollection<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>> apply(PCollection<T> input) {
+ @SuppressWarnings("unchecked")
+ Coder<W> windowCoder = (Coder<W>)
+ input.getWindowingStrategy().getWindowFn().windowCoder();
+ PCollection<KV<Integer, KV<W, WindowedValue<T>>>> rval =
+ input.apply(ParDo.of(
+ new UseWindowHashAsKeyAndWindowAsSortKeyDoFn<T, W>(ismCoderForHash)));
+ rval.setCoder(
+ KvCoder.of(
+ VarIntCoder.of(),
+ KvCoder.of(windowCoder,
+ FullWindowedValueCoder.of(input.getCoder(), windowCoder))));
+ return rval.apply(new GroupByKeyAndSortValuesOnly<Integer, W, WindowedValue<T>>());
+ }
+ }
+
+ /**
+ * Specialized implementation for
+ * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} for the
+ * Dataflow runner in batch mode.
+ *
+ * <p>Creates a set of files in the {@link IsmFormat} sharded by the hash of the windows
+ * byte representation and with records having:
+ * <ul>
+ * <li>Key 1: Window</li>
+ * <li>Value: Windowed value</li>
+ * </ul>
+ */
+ static class BatchViewAsSingleton<T>
+ extends PTransform<PCollection<T>, PCollectionView<T>> {
+
+ /**
+ * A {@link DoFn} that outputs {@link IsmRecord}s. These records are structured as follows:
+ * <ul>
+ * <li>Key 1: Window
+ * <li>Value: Windowed value
+ * </ul>
+ */
+ static class IsmRecordForSingularValuePerWindowDoFn<T, W extends BoundedWindow>
+ extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
+ IsmRecord<WindowedValue<T>>> {
+
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ Iterator<KV<W, WindowedValue<T>>> iterator = c.element().getValue().iterator();
+ while (iterator.hasNext()) {
+ KV<W, WindowedValue<T>> next = iterator.next();
+ c.output(
+ IsmRecord.of(
+ ImmutableList.of(next.getKey()), next.getValue()));
+ }
+ }
+ }
+
+ private final DataflowPipelineRunner runner;
+ private final View.AsSingleton<T> transform;
+ /**
+ * Builds an instance of this class from the overridden transform.
+ */
+ @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
+ public BatchViewAsSingleton(DataflowPipelineRunner runner, View.AsSingleton<T> transform) {
+ this.runner = runner;
+ this.transform = transform;
+ }
+
+ @Override
+ public PCollectionView<T> apply(PCollection<T> input) {
+ return BatchViewAsSingleton.<T, T, T, BoundedWindow>applyForSingleton(
+ runner,
+ input,
+ new IsmRecordForSingularValuePerWindowDoFn<T, BoundedWindow>(),
+ transform.hasDefaultValue(),
+ transform.defaultValue(),
+ input.getCoder());
+ }
+
+ static <T, FinalT, ViewT, W extends BoundedWindow> PCollectionView<ViewT>
+ applyForSingleton(
+ DataflowPipelineRunner runner,
+ PCollection<T> input,
+ DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
+ IsmRecord<WindowedValue<FinalT>>> doFn,
+ boolean hasDefault,
+ FinalT defaultValue,
+ Coder<FinalT> defaultValueCoder) {
+
+ @SuppressWarnings("unchecked")
+ Coder<W> windowCoder = (Coder<W>)
+ input.getWindowingStrategy().getWindowFn().windowCoder();
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ PCollectionView<ViewT> view =
+ (PCollectionView<ViewT>) PCollectionViews.<FinalT, W>singletonView(
+ input.getPipeline(),
+ (WindowingStrategy) input.getWindowingStrategy(),
+ hasDefault,
+ defaultValue,
+ defaultValueCoder);
+
+ IsmRecordCoder<WindowedValue<FinalT>> ismCoder =
+ coderForSingleton(windowCoder, defaultValueCoder);
+
+ PCollection<IsmRecord<WindowedValue<FinalT>>> reifiedPerWindowAndSorted = input
+ .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
+ .apply(ParDo.of(doFn));
+ reifiedPerWindowAndSorted.setCoder(ismCoder);
+
+ runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
+ return reifiedPerWindowAndSorted.apply(
+ CreatePCollectionView.<IsmRecord<WindowedValue<FinalT>>, ViewT>of(view));
+ }
+
+ @Override
+ protected String getKindString() {
+ return "BatchViewAsSingleton";
+ }
+
+ static <T> IsmRecordCoder<WindowedValue<T>> coderForSingleton(
+ Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
+ return IsmRecordCoder.of(
+ 1, // We hash using only the window
+ 0, // There are no metadata records
+ ImmutableList.<Coder<?>>of(windowCoder),
+ FullWindowedValueCoder.of(valueCoder, windowCoder));
+ }
+ }
+
+ /**
+ * Specialized implementation for
+ * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the
+ * Dataflow runner in batch mode.
+ *
+ * <p>Creates a set of {@code Ism} files sharded by the hash of the windows byte representation
+ * and with records having:
+ * <ul>
+ * <li>Key 1: Window</li>
+ * <li>Key 2: Index offset within window</li>
+ * <li>Value: Windowed value</li>
+ * </ul>
+ */
+ static class BatchViewAsIterable<T>
+ extends PTransform<PCollection<T>, PCollectionView<Iterable<T>>> {
+
+ private final DataflowPipelineRunner runner;
+ /**
+ * Builds an instance of this class from the overridden transform.
+ */
+ @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
+ public BatchViewAsIterable(DataflowPipelineRunner runner, View.AsIterable<T> transform) {
+ this.runner = runner;
+ }
+
+ @Override
+ public PCollectionView<Iterable<T>> apply(PCollection<T> input) {
+ PCollectionView<Iterable<T>> view = PCollectionViews.iterableView(
+ input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
+ return BatchViewAsList.applyForIterableLike(runner, input, view);
+ }
+ }
+
+ /**
+ * Specialized implementation for
+ * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the
+ * Dataflow runner in batch mode.
+ *
+ * <p>Creates a set of {@code Ism} files sharded by the hash of the window's byte representation
+ * and with records having:
+ * <ul>
+ * <li>Key 1: Window</li>
+ * <li>Key 2: Index offset within window</li>
+ * <li>Value: Windowed value</li>
+ * </ul>
+ */
+ static class BatchViewAsList<T>
+ extends PTransform<PCollection<T>, PCollectionView<List<T>>> {
+ /**
+ * A {@link DoFn} which creates {@link IsmRecord}s assuming that each element is within the
+ * global window. Each {@link IsmRecord} has
+ * <ul>
+ * <li>Key 1: Global window</li>
+ * <li>Key 2: Index offset within window</li>
+ * <li>Value: Windowed value</li>
+ * </ul>
+ */
+ @SystemDoFnInternal
+ static class ToIsmRecordForGlobalWindowDoFn<T>
+ extends DoFn<T, IsmRecord<WindowedValue<T>>> {
+
+ long indexInBundle;
+ @Override
+ public void startBundle(Context c) throws Exception {
+ indexInBundle = 0;
+ }
+
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ c.output(IsmRecord.of(
+ ImmutableList.of(GlobalWindow.INSTANCE, indexInBundle),
+ WindowedValue.of(
+ c.element(),
+ c.timestamp(),
+ GlobalWindow.INSTANCE,
+ c.pane())));
+ indexInBundle += 1;
+ }
+ }
+
+ /**
+ * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
+ * to locate the window boundaries. The {@link IsmRecord} has:
+ * <ul>
+ * <li>Key 1: Window</li>
+ * <li>Key 2: Index offset within window</li>
+ * <li>Value: Windowed value</li>
+ * </ul>
+ */
+ @SystemDoFnInternal
+ static class ToIsmRecordForNonGlobalWindowDoFn<T, W extends BoundedWindow>
+ extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>,
+ IsmRecord<WindowedValue<T>>> {
+
+ private final Coder<W> windowCoder;
+ ToIsmRecordForNonGlobalWindowDoFn(Coder<W> windowCoder) {
+ this.windowCoder = windowCoder;
+ }
+
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ long elementsInWindow = 0;
+ Optional<Object> previousWindowStructuralValue = Optional.absent();
+ for (KV<W, WindowedValue<T>> value : c.element().getValue()) {
+ Object currentWindowStructuralValue = windowCoder.structuralValue(value.getKey());
+ // Compare to see if this is a new window so we can reset the index counter i
+ if (previousWindowStructuralValue.isPresent()
+ && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
+ // Reset i since we have a new window.
+ elementsInWindow = 0;
+ }
+ c.output(IsmRecord.of(
+ ImmutableList.of(value.getKey(), elementsInWindow),
+ value.getValue()));
+ previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
+ elementsInWindow += 1;
+ }
+ }
+ }
+
+ private final DataflowPipelineRunner runner;
+ /**
+ * Builds an instance of this class from the overridden transform.
+ */
+ @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
+ public BatchViewAsList(DataflowPipelineRunner runner, View.AsList<T> transform) {
+ this.runner = runner;
+ }
+
+ @Override
+ public PCollectionView<List<T>> apply(PCollection<T> input) {
+ PCollectionView<List<T>> view = PCollectionViews.listView(
+ input.getPipeline(), input.getWindowingStrategy(), input.getCoder());
+ return applyForIterableLike(runner, input, view);
+ }
+
+ static <T, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForIterableLike(
+ DataflowPipelineRunner runner,
+ PCollection<T> input,
+ PCollectionView<ViewT> view) {
+
+ @SuppressWarnings("unchecked")
+ Coder<W> windowCoder = (Coder<W>)
+ input.getWindowingStrategy().getWindowFn().windowCoder();
+
+ IsmRecordCoder<WindowedValue<T>> ismCoder = coderForListLike(windowCoder, input.getCoder());
+
+ // If we are working in the global window, we do not need to do a GBK using the window
+ // as the key since all the elements of the input PCollection are already such.
+ // We just reify the windowed value while converting them to IsmRecords and generating
+ // an index based upon where we are within the bundle. Each bundle
+ // maps to one file exactly.
+ if (input.getWindowingStrategy().getWindowFn() instanceof GlobalWindows) {
+ PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted =
+ input.apply(ParDo.of(new ToIsmRecordForGlobalWindowDoFn<T>()));
+ reifiedPerWindowAndSorted.setCoder(ismCoder);
+
+ runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
+ return reifiedPerWindowAndSorted.apply(
+ CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
+ }
+
+ PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted = input
+ .apply(new GroupByWindowHashAsKeyAndWindowAsSortKey<T, W>(ismCoder))
+ .apply(ParDo.of(new ToIsmRecordForNonGlobalWindowDoFn<T, W>(windowCoder)));
+ reifiedPerWindowAndSorted.setCoder(ismCoder);
+
+ runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
+ return reifiedPerWindowAndSorted.apply(
+ CreatePCollectionView.<IsmRecord<WindowedValue<T>>, ViewT>of(view));
+ }
+
+ @Override
+ protected String getKindString() {
+ return "BatchViewAsList";
+ }
+
+ static <T> IsmRecordCoder<WindowedValue<T>> coderForListLike(
+ Coder<? extends BoundedWindow> windowCoder, Coder<T> valueCoder) {
+ // TODO: swap to use a variable length long coder which has values which compare
+ // the same as their byte representation compare lexicographically within the key coder
+ return IsmRecordCoder.of(
+ 1, // We hash using only the window
+ 0, // There are no metadata records
+ ImmutableList.of(windowCoder, BigEndianLongCoder.of()),
+ FullWindowedValueCoder.of(valueCoder, windowCoder));
+ }
+ }
+
+ /**
+ * Specialized implementation for
+ * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} for the
+ * Dataflow runner in batch mode.
+ *
+ * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
+ * representation. Each record is structured as follows:
+ * <ul>
+ * <li>Key 1: User key K</li>
+ * <li>Key 2: Window</li>
+ * <li>Key 3: 0L (constant)</li>
+ * <li>Value: Windowed value</li>
+ * </ul>
+ *
+ * <p>Alongside the data records, there are the following metadata records:
+ * <ul>
+ * <li>Key 1: Metadata Key</li>
+ * <li>Key 2: Window</li>
+ * <li>Key 3: Index [0, size of map]</li>
+ * <li>Value: variable length long byte representation of size of map if index is 0,
+ * otherwise the byte representation of a key</li>
+ * </ul>
+ * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
+ * {@code [META, Window, i]} for {@code i} in {@code [1, size of map]} stores a the users key.
+ * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
+ * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
+ * {@code [1, size of map]}.
+ *
+ * <p>Note that in the case of a non-deterministic key coder, we fallback to using
+ * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing
+ * a warning to users to specify a deterministic key coder.
+ */
+ static class BatchViewAsMap<K, V>
+ extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, V>>> {
+
+ /**
+ * A {@link DoFn} which groups elements by window boundaries. For each group,
+ * the group of elements is transformed into a {@link TransformedMap}.
+ * The transformed {@code Map<K, V>} is backed by a {@code Map<K, WindowedValue<V>>}
+ * and contains a function {@code WindowedValue<V> -> V}.
+ *
+ * <p>Outputs {@link IsmRecord}s having:
+ * <ul>
+ * <li>Key 1: Window</li>
+ * <li>Value: Transformed map containing a transform that removes the encapsulation
+ * of the window around each value,
+ * {@code Map<K, WindowedValue<V>> -> Map<K, V>}.</li>
+ * </ul>
+ */
+ static class ToMapDoFn<K, V, W extends BoundedWindow>
+ extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
+ IsmRecord<WindowedValue<TransformedMap<K,
+ WindowedValue<V>,
+ V>>>> {
+
+ private final Coder<W> windowCoder;
+ ToMapDoFn(Coder<W> windowCoder) {
+ this.windowCoder = windowCoder;
+ }
+
+ @Override
+ public void processElement(ProcessContext c)
+ throws Exception {
+ Optional<Object> previousWindowStructuralValue = Optional.absent();
+ Optional<W> previousWindow = Optional.absent();
+ Map<K, WindowedValue<V>> map = new HashMap<>();
+ for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
+ Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
+ if (previousWindowStructuralValue.isPresent()
+ && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
+ // Construct the transformed map containing all the elements since we
+ // are at a window boundary.
+ c.output(IsmRecord.of(
+ ImmutableList.of(previousWindow.get()),
+ valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
+ map = new HashMap<>();
+ }
+
+ // Verify that the user isn't trying to insert the same key multiple times.
+ checkState(!map.containsKey(kv.getValue().getValue().getKey()),
+ "Multiple values [%s, %s] found for single key [%s] within window [%s].",
+ map.get(kv.getValue().getValue().getKey()),
+ kv.getValue().getValue().getValue(),
+ kv.getKey());
+ map.put(kv.getValue().getValue().getKey(),
+ kv.getValue().withValue(kv.getValue().getValue().getValue()));
+ previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
+ previousWindow = Optional.of(kv.getKey());
+ }
+
+ // The last value for this hash is guaranteed to be at a window boundary
+ // so we output a transformed map containing all the elements since the last
+ // window boundary.
+ c.output(IsmRecord.of(
+ ImmutableList.of(previousWindow.get()),
+ valueInEmptyWindows(new TransformedMap<>(WindowedValueToValue.<V>of(), map))));
+ }
+ }
+
+ private final DataflowPipelineRunner runner;
+ /**
+ * Builds an instance of this class from the overridden transform.
+ */
+ @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
+ public BatchViewAsMap(DataflowPipelineRunner runner, View.AsMap<K, V> transform) {
+ this.runner = runner;
+ }
+
+ @Override
+ public PCollectionView<Map<K, V>> apply(PCollection<KV<K, V>> input) {
+ return this.<BoundedWindow>applyInternal(input);
+ }
+
+ private <W extends BoundedWindow> PCollectionView<Map<K, V>>
+ applyInternal(PCollection<KV<K, V>> input) {
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+ try {
+ PCollectionView<Map<K, V>> view = PCollectionViews.mapView(
+ input.getPipeline(), input.getWindowingStrategy(), inputCoder);
+ return BatchViewAsMultimap.applyForMapLike(runner, input, view, true /* unique keys */);
+ } catch (NonDeterministicException e) {
+ runner.recordViewUsesNonDeterministicKeyCoder(this);
+
+ // Since the key coder is not deterministic, we convert the map into a singleton
+ // and return a singleton view equivalent.
+ return applyForSingletonFallback(input);
+ }
+ }
+
+ @Override
+ protected String getKindString() {
+ return "BatchViewAsMap";
+ }
+
+ /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
+ private <W extends BoundedWindow> PCollectionView<Map<K, V>>
+ applyForSingletonFallback(PCollection<KV<K, V>> input) {
+ @SuppressWarnings("unchecked")
+ Coder<W> windowCoder = (Coder<W>)
+ input.getWindowingStrategy().getWindowFn().windowCoder();
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ Coder<Function<WindowedValue<V>, V>> transformCoder =
+ (Coder) SerializableCoder.of(WindowedValueToValue.class);
+
+ Coder<TransformedMap<K, WindowedValue<V>, V>> finalValueCoder =
+ TransformedMapCoder.of(
+ transformCoder,
+ MapCoder.of(
+ inputCoder.getKeyCoder(),
+ FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder)));
+
+ TransformedMap<K, WindowedValue<V>, V> defaultValue = new TransformedMap<>(
+ WindowedValueToValue.<V>of(),
+ ImmutableMap.<K, WindowedValue<V>>of());
+
+ return BatchViewAsSingleton.<KV<K, V>,
+ TransformedMap<K, WindowedValue<V>, V>,
+ Map<K, V>,
+ W> applyForSingleton(
+ runner,
+ input,
+ new ToMapDoFn<K, V, W>(windowCoder),
+ true,
+ defaultValue,
+ finalValueCoder);
+ }
+ }
+
+ /**
+ * Specialized implementation for
+ * {@link org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the
+ * Dataflow runner in batch mode.
+ *
+ * <p>Creates a set of {@code Ism} files sharded by the hash of the key's byte
+ * representation. Each record is structured as follows:
+ * <ul>
+ * <li>Key 1: User key K</li>
+ * <li>Key 2: Window</li>
+ * <li>Key 3: Index offset for a given key and window.</li>
+ * <li>Value: Windowed value</li>
+ * </ul>
+ *
+ * <p>Alongside the data records, there are the following metadata records:
+ * <ul>
+ * <li>Key 1: Metadata Key</li>
+ * <li>Key 2: Window</li>
+ * <li>Key 3: Index [0, size of map]</li>
+ * <li>Value: variable length long byte representation of size of map if index is 0,
+ * otherwise the byte representation of a key</li>
+ * </ul>
+ * The {@code [META, Window, 0]} record stores the number of unique keys per window, while
+ * {@code [META, Window, i]} for {@code i} in {@code [1, size of map]} stores a the users key.
+ * This allows for one to access the size of the map by looking at {@code [META, Window, 0]}
+ * and iterate over all the keys by accessing {@code [META, Window, i]} for {@code i} in
+ * {@code [1, size of map]}.
+ *
+ * <p>Note that in the case of a non-deterministic key coder, we fallback to using
+ * {@link org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing
+ * a warning to users to specify a deterministic key coder.
+ */
+ static class BatchViewAsMultimap<K, V>
+ extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, Iterable<V>>>> {
+ /**
+ * A {@link PTransform} that groups elements by the hash of window's byte representation
+ * if the input {@link PCollection} is not within the global window. Otherwise by the hash
+ * of the window and key's byte representation. This {@link PTransform} also sorts
+ * the values by the combination of the window and key's byte representations.
+ */
+ private static class GroupByKeyHashAndSortByKeyAndWindow<K, V, W extends BoundedWindow>
+ extends PTransform<PCollection<KV<K, V>>,
+ PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>> {
+
+ @SystemDoFnInternal
+ private static class GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>
+ extends DoFn<KV<K, V>, KV<Integer, KV<KV<K, W>, WindowedValue<V>>>>
+ implements DoFn.RequiresWindowAccess {
+
+ private final IsmRecordCoder<?> coder;
+ private GroupByKeyHashAndSortByKeyAndWindowDoFn(IsmRecordCoder<?> coder) {
+ this.coder = coder;
+ }
+
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ @SuppressWarnings("unchecked")
+ W window = (W) c.window();
+
+ c.output(
+ KV.of(coder.hash(ImmutableList.of(c.element().getKey())),
+ KV.of(KV.of(c.element().getKey(), window),
+ WindowedValue.of(
+ c.element().getValue(),
+ c.timestamp(),
+ (BoundedWindow) window,
+ c.pane()))));
+ }
+ }
+
+ private final IsmRecordCoder<?> coder;
+ public GroupByKeyHashAndSortByKeyAndWindow(IsmRecordCoder<?> coder) {
+ this.coder = coder;
+ }
+
+ @Override
+ public PCollection<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>>
+ apply(PCollection<KV<K, V>> input) {
+
+ @SuppressWarnings("unchecked")
+ Coder<W> windowCoder = (Coder<W>)
+ input.getWindowingStrategy().getWindowFn().windowCoder();
+ @SuppressWarnings("unchecked")
+ KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
+
+ PCollection<KV<Integer, KV<KV<K, W>, WindowedValue<V>>>> keyedByHash;
+ keyedByHash = input.apply(
+ ParDo.of(new GroupByKeyHashAndSortByKeyAndWindowDoFn<K, V, W>(coder)));
+ keyedByHash.setCoder(
+ KvCoder.of(
+ VarIntCoder.of(),
+ KvCoder.of(KvCoder.of(inputCoder.getKeyCoder(), windowCoder),
+ FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
+
+ return keyedByHash.apply(
+ new GroupByKeyAndSortValuesOnly<Integer, KV<K, W>, WindowedValue<V>>());
+ }
+ }
+
+ /**
+ * A {@link DoFn} which creates {@link IsmRecord}s comparing successive elements windows
+ * and keys to locate window and key boundaries. The main output {@link IsmRecord}s have:
+ * <ul>
+ * <li>Key 1: Window</li>
+ * <li>Key 2: User key K</li>
+ * <li>Key 3: Index offset for a given key and window.</li>
+ * <li>Value: Windowed value</li>
+ * </ul>
+ *
+ * <p>Additionally, we output all the unique keys per window seen to {@code outputForEntrySet}
+ * and the unique key count per window to {@code outputForSize}.
+ *
+ * <p>Finally, if this DoFn has been requested to perform unique key checking, it will
+ * throw an {@link IllegalStateException} if more than one key per window is found.
+ */
+ static class ToIsmRecordForMapLikeDoFn<K, V, W extends BoundedWindow>
+ extends DoFn<KV<Integer, Iterable<KV<KV<K, W>, WindowedValue<V>>>>,
+ IsmRecord<WindowedValue<V>>> {
+
+ private final TupleTag<KV<Integer, KV<W, Long>>> outputForSize;
+ private final TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet;
+ private final Coder<W> windowCoder;
+ private final Coder<K> keyCoder;
+ private final IsmRecordCoder<WindowedValue<V>> ismCoder;
+ private final boolean uniqueKeysExpected;
+ ToIsmRecordForMapLikeDoFn(
+ TupleTag<KV<Integer, KV<W, Long>>> outputForSize,
+ TupleTag<KV<Integer, KV<W, K>>> outputForEntrySet,
+ Coder<W> windowCoder,
+ Coder<K> keyCoder,
+ IsmRecordCoder<WindowedValue<V>> ismCoder,
+ boolean uniqueKeysExpected) {
+ this.outputForSize = outputForSize;
+ this.outputForEntrySet = outputForEntrySet;
+ this.windowCoder = windowCoder;
+ this.keyCoder = keyCoder;
+ this.ismCoder = ismCoder;
+ this.uniqueKeysExpected = uniqueKeysExpected;
+ }
+
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ long currentKeyIndex = 0;
+ // We use one based indexing while counting
+ long currentUniqueKeyCounter = 1;
+ Iterator<KV<KV<K, W>, WindowedValue<V>>> iterator = c.element().getValue().iterator();
+
+ KV<KV<K, W>, WindowedValue<V>> currentValue = iterator.next();
+ Object currentKeyStructuralValue =
+ keyCoder.structuralValue(currentValue.getKey().getKey());
+ Object currentWindowStructuralValue =
+ windowCoder.structuralValue(currentValue.getKey().getValue());
+
+ while (iterator.hasNext()) {
+ KV<KV<K, W>, WindowedValue<V>> nextValue = iterator.next();
+ Object nextKeyStructuralValue =
+ keyCoder.structuralValue(nextValue.getKey().getKey());
+ Object nextWindowStructuralValue =
+ windowCoder.structuralValue(nextValue.getKey().getValue());
+
+ outputDataRecord(c, currentValue, currentKeyIndex);
+
+ final long nextKeyIndex;
+ final long nextUniqueKeyCounter;
+
+ // Check to see if its a new window
+ if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
+ // The next value is a new window, so we output for size the number of unique keys
+ // seen and the last key of the window. We also reset the next key index the unique
+ // key counter.
+ outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
+ outputMetadataRecordForEntrySet(c, currentValue);
+
+ nextKeyIndex = 0;
+ nextUniqueKeyCounter = 1;
+ } else if (!currentKeyStructuralValue.equals(nextKeyStructuralValue)){
+ // It is a new key within the same window so output the key for the entry set,
+ // reset the key index and increase the count of unique keys seen within this window.
+ outputMetadataRecordForEntrySet(c, currentValue);
+
+ nextKeyIndex = 0;
+ nextUniqueKeyCounter = currentUniqueKeyCounter + 1;
+ } else if (!uniqueKeysExpected) {
+ // It is not a new key so we don't have to output the number of elements in this
+ // window or increase the unique key counter. All we do is increase the key index.
+
+ nextKeyIndex = currentKeyIndex + 1;
+ nextUniqueKeyCounter = currentUniqueKeyCounter;
+ } else {
+ throw new IllegalStateException(String.format(
+ "Unique keys are expected but found key %s with values %s and %s in window %s.",
+ currentValue.getKey().getKey(),
+ currentValue.getValue().getValue(),
+ nextValue.getValue().getValue(),
+ currentValue.getKey().getValue()));
+ }
+
+ currentValue = nextValue;
+ currentWindowStructuralValue = nextWindowStructuralValue;
+ currentKeyStructuralValue = nextKeyStructuralValue;
+ currentKeyIndex = nextKeyIndex;
+ currentUniqueKeyCounter = nextUniqueKeyCounter;
+ }
+
+ outputDataRecord(c, currentValue, currentKeyIndex);
+ outputMetadataRecordForSize(c, currentValue, currentUniqueKeyCounter);
+ // The last value for this hash is guaranteed to be at a window boundary
+ // so we output a record with the number of unique keys seen.
+ outputMetadataRecordForEntrySet(c, currentValue);
+ }
+
+ /** This outputs the data record. */
+ private void outputDataRecord(
+ ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long keyIndex) {
+ IsmRecord<WindowedValue<V>> ismRecord = IsmRecord.of(
+ ImmutableList.of(
+ value.getKey().getKey(),
+ value.getKey().getValue(),
+ keyIndex),
+ value.getValue());
+ c.output(ismRecord);
+ }
+
+ /**
+ * This outputs records which will be used to compute the number of keys for a given window.
+ */
+ private void outputMetadataRecordForSize(
+ ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value, long uniqueKeyCount) {
+ c.sideOutput(outputForSize,
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
+ value.getKey().getValue())),
+ KV.of(value.getKey().getValue(), uniqueKeyCount)));
+ }
+
+ /** This outputs records which will be used to construct the entry set. */
+ private void outputMetadataRecordForEntrySet(
+ ProcessContext c, KV<KV<K, W>, WindowedValue<V>> value) {
+ c.sideOutput(outputForEntrySet,
+ KV.of(ismCoder.hash(ImmutableList.of(IsmFormat.getMetadataKey(),
+ value.getKey().getValue())),
+ KV.of(value.getKey().getValue(), value.getKey().getKey())));
+ }
+ }
+
+ /**
+ * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window of:
+ * <ul>
+ * <li>Key 1: META key</li>
+ * <li>Key 2: window</li>
+ * <li>Key 3: 0L (constant)</li>
+ * <li>Value: sum of values for window</li>
+ * </ul>
+ *
+ * <p>This {@link DoFn} is meant to be used to compute the number of unique keys
+ * per window for map and multimap side inputs.
+ */
+ static class ToIsmMetadataRecordForSizeDoFn<K, V, W extends BoundedWindow>
+ extends DoFn<KV<Integer, Iterable<KV<W, Long>>>, IsmRecord<WindowedValue<V>>> {
+ private final Coder<W> windowCoder;
+ ToIsmMetadataRecordForSizeDoFn(Coder<W> windowCoder) {
+ this.windowCoder = windowCoder;
+ }
+
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ Iterator<KV<W, Long>> iterator = c.element().getValue().iterator();
+ KV<W, Long> currentValue = iterator.next();
+ Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
+ long size = 0;
+ while (iterator.hasNext()) {
+ KV<W, Long> nextValue = iterator.next();
+ Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
+
+ size += currentValue.getValue();
+ if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
+ c.output(IsmRecord.<WindowedValue<V>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
+ CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
+ size = 0;
+ }
+
+ currentValue = nextValue;
+ currentWindowStructuralValue = nextWindowStructuralValue;
+ }
+
+ size += currentValue.getValue();
+ // Output the final value since it is guaranteed to be on a window boundary.
+ c.output(IsmRecord.<WindowedValue<V>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), 0L),
+ CoderUtils.encodeToByteArray(VarLongCoder.of(), size)));
+ }
+ }
+
+ /**
+ * A {@link DoFn} which outputs a metadata {@link IsmRecord} per window and key pair of:
+ * <ul>
+ * <li>Key 1: META key</li>
+ * <li>Key 2: window</li>
+ * <li>Key 3: index offset (1-based index)</li>
+ * <li>Value: key</li>
+ * </ul>
+ *
+ * <p>This {@link DoFn} is meant to be used to output index to key records
+ * per window for map and multimap side inputs.
+ */
+ static class ToIsmMetadataRecordForKeyDoFn<K, V, W extends BoundedWindow>
+ extends DoFn<KV<Integer, Iterable<KV<W, K>>>, IsmRecord<WindowedValue<V>>> {
+
+ private final Coder<K> keyCoder;
+ private final Coder<W> windowCoder;
+ ToIsmMetadataRecordForKeyDoFn(Coder<K> keyCoder, Coder<W> windowCoder) {
+ this.keyCoder = keyCoder;
+ this.windowCoder = windowCoder;
+ }
+
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ Iterator<KV<W, K>> iterator = c.element().getValue().iterator();
+ KV<W, K> currentValue = iterator.next();
+ Object currentWindowStructuralValue = windowCoder.structuralValue(currentValue.getKey());
+ long elementsInWindow = 1;
+ while (iterator.hasNext()) {
+ KV<W, K> nextValue = iterator.next();
+ Object nextWindowStructuralValue = windowCoder.structuralValue(nextValue.getKey());
+
+ c.output(IsmRecord.<WindowedValue<V>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
+ CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
+ elementsInWindow += 1;
+
+ if (!currentWindowStructuralValue.equals(nextWindowStructuralValue)) {
+ elementsInWindow = 1;
+ }
+
+ currentValue = nextValue;
+ currentWindowStructuralValue = nextWindowStructuralValue;
+ }
+
+ // Output the final value since it is guaranteed to be on a window boundary.
+ c.output(IsmRecord.<WindowedValue<V>>meta(
+ ImmutableList.of(IsmFormat.getMetadataKey(), currentValue.getKey(), elementsInWindow),
+ CoderUtils.encodeToByteArray(keyCoder, currentValue.getValue())));
+ }
+ }
+
+ /**
+ * A {@link DoFn} which partitions sets of elements by window boundaries. Within each
+ * partition, the set of elements is transformed into a {@link TransformedMap}.
+ * The transformed {@code Map<K, Iterable<V>>} is backed by a
+ * {@code Map<K, Iterable<WindowedValue<V>>>} and contains a function
+ * {@code Iterable<WindowedValue<V>> -> Iterable<V>}.
+ *
+ * <p>Outputs {@link IsmRecord}s having:
+ * <ul>
+ * <li>Key 1: Window</li>
+ * <li>Value: Transformed map containing a transform that removes the encapsulation
+ * of the window around each value,
+ * {@code Map<K, Iterable<WindowedValue<V>>> -> Map<K, Iterable<V>>}.</li>
+ * </ul>
+ */
+ static class ToMultimapDoFn<K, V, W extends BoundedWindow>
+ extends DoFn<KV<Integer, Iterable<KV<W, WindowedValue<KV<K, V>>>>>,
+ IsmRecord<WindowedValue<TransformedMap<K,
+ Iterable<WindowedValue<V>>,
+ Iterable<V>>>>> {
+
+ private final Coder<W> windowCoder;
+ ToMultimapDoFn(Coder<W> windowCoder) {
+ this.windowCoder = windowCoder;
+ }
+
+ @Override
+ public void processElement(ProcessContext c)
+ throws Exception {
+ Optional<Object> previousWindowStructuralValue = Optional.absent();
+ Optional<W> previousWindow = Optional.absent();
+ Multimap<K, WindowedValue<V>> multimap = HashMultimap.create();
+ for (KV<W, WindowedValue<KV<K, V>>> kv : c.element().getValue()) {
+ Object currentWindowStructuralValue = windowCoder.structuralValue(kv.getKey());
+ if (previousWindowStructuralValue.isPresent()
+ && !previousWindowStructuralValue.get().equals(currentWindowStructuralValue)) {
+ // Construct the transformed map containing all the elements since we
+ // are at a window boundary.
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
+ c.output(IsmRecord.<WindowedValue<TransformedMap<K,
+ Iterable<WindowedValue<V>>,
+ Iterable<V>>>>of(
+ ImmutableList.of(previousWindow.get()),
+ valueInEmptyWindows(
+ new TransformedMap<>(
+ IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
+ multimap = HashMultimap.create();
+ }
+
+ multimap.put(kv.getValue().getValue().getKey(),
+ kv.getValue().withValue(kv.getValue().getValue().getValue()));
+ previousWindowStructuralValue = Optional.of(currentWindowStructuralValue);
+ previousWindow = Optional.of(kv.getKey());
+ }
+
+ // The last value for this hash is guaranteed to be at a window boundary
+ // so we output a transformed map containing all the elements since the last
+ // window boundary.
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ Map<K, Iterable<WindowedValue<V>>> resultMap = (Map) multimap.asMap();
+ c.output(IsmRecord.<WindowedValue<TransformedMap<K,
+ Iterable<WindowedValue<V>>,
+ Iterable<V>>>>of(
+ ImmutableList.of(previousWindow.get()),
+ valueInEmptyWindows(
+ new TransformedMap<>(IterableWithWindowedValuesToIterable.<V>of(), resultMap))));
+ }
+ }
+
+ private final DataflowPipelineRunner runner;
+ /**
+ * Builds an instance of this class from the overridden transform.
+ */
+ @SuppressWarnings("unused") // used via reflection in DataflowPipelineRunner#apply()
+ public BatchViewAsMultimap(DataflowPipelineRunner runner, View.AsMultimap<K, V> transform) {
+ this.runner = runner;
+ }
+
+ @Override
+ public PCollectionView<Map<K, Iterable<V>>> apply(PCollection<KV<K, V>> input) {
+ return this.<BoundedWindow>applyInternal(input);
+ }
+
+ private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
+ applyInternal(PCollection<KV<K, V>> input) {
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+ try {
+ PCollectionView<Map<K, Iterable<V>>> view = PCollectionViews.multimapView(
+ input.getPipeline(), input.getWindowingStrategy(), inputCoder);
+
+ return applyForMapLike(runner, input, view, false /* unique keys not expected */);
+ } catch (NonDeterministicException e) {
+ runner.recordViewUsesNonDeterministicKeyCoder(this);
+
+ // Since the key coder is not deterministic, we convert the map into a singleton
+ // and return a singleton view equivalent.
+ return applyForSingletonFallback(input);
+ }
+ }
+
+ /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
+ private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>>
+ applyForSingletonFallback(PCollection<KV<K, V>> input) {
+ @SuppressWarnings("unchecked")
+ Coder<W> windowCoder = (Coder<W>)
+ input.getWindowingStrategy().getWindowFn().windowCoder();
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ Coder<Function<Iterable<WindowedValue<V>>, Iterable<V>>> transformCoder =
+ (Coder) SerializableCoder.of(IterableWithWindowedValuesToIterable.class);
+
+ Coder<TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>> finalValueCoder =
+ TransformedMapCoder.of(
+ transformCoder,
+ MapCoder.of(
+ inputCoder.getKeyCoder(),
+ IterableCoder.of(
+ FullWindowedValueCoder.of(inputCoder.getValueCoder(), windowCoder))));
+
+ TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>> defaultValue =
+ new TransformedMap<>(
+ IterableWithWindowedValuesToIterable.<V>of(),
+ ImmutableMap.<K, Iterable<WindowedValue<V>>>of());
+
+ return BatchViewAsSingleton.<KV<K, V>,
+ TransformedMap<K, Iterable<WindowedValue<V>>, Iterable<V>>,
+ Map<K, Iterable<V>>,
+ W> applyForSingleton(
+ runner,
+ input,
+ new ToMultimapDoFn<K, V, W>(windowCoder),
+ true,
+ defaultValue,
+ finalValueCoder);
+ }
+
+ private static <K, V, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForMapLike(
+ DataflowPipelineRunner runner,
+ PCollection<KV<K, V>> input,
+ PCollectionView<ViewT> view,
+ boolean uniqueKeysExpected) throws NonDeterministicException {
+
+ @SuppressWarnings("unchecked")
+ Coder<W> windowCoder = (Coder<W>)
+ input.getWindowingStrategy().getWindowFn().windowCoder();
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ KvCoder<K, V> inputCoder = (KvCoder) input.getCoder();
+
+ // If our key coder is deterministic, we can use the key portion of each KV
+ // part of a composite key containing the window , key and index.
+ inputCoder.getKeyCoder().verifyDeterministic();
+
+ IsmRecordCoder<WindowedValue<V>> ismCoder =
+ coderForMapLike(windowCoder, inputCoder.getKeyCoder(), inputCoder.getValueCoder());
+
+ // Create the various output tags representing the main output containing the data stream
+ // and the side outputs containing the metadata about the size and entry set.
+ TupleTag<IsmRecord<WindowedValue<V>>> mainOutputTag = new TupleTag<>();
+ TupleTag<KV<Integer, KV<W, Long>>> outputForSizeTag = new TupleTag<>();
+ TupleTag<KV<Integer, KV<W, K>>> outputForEntrySetTag = new TupleTag<>();
+
+ // Process all the elements grouped by key hash, and sorted by key and then window
+ // outputting to all the outputs defined above.
+ PCollectionTuple outputTuple = input
+ .apply("GBKaSVForData", new GroupByKeyHashAndSortByKeyAndWindow<K, V, W>(ismCoder))
+ .apply(ParDo.of(new ToIsmRecordForMapLikeDoFn<K, V, W>(
+ outputForSizeTag, outputForEntrySetTag,
+ windowCoder, inputCoder.getKeyCoder(), ismCoder, uniqueKeysExpected))
+ .withOutputTags(mainOutputTag,
+ TupleTagList.of(
+ ImmutableList.<TupleTag<?>>of(outputForSizeTag,
+ outputForEntrySetTag))));
+
+ // Set the coder on the main data output.
+ PCollection<IsmRecord<WindowedValue<V>>> perHashWithReifiedWindows =
+ outputTuple.get(mainOutputTag);
+ perHashWithReifiedWindows.setCoder(ismCoder);
+
+ // Set the coder on the metadata output for size and process the entries
+ // producing a [META, Window, 0L] record per window storing the number of unique keys
+ // for each window.
+ PCollection<KV<Integer, KV<W, Long>>> outputForSize = outputTuple.get(outputForSizeTag);
+ outputForSize.setCoder(
+ KvCoder.of(VarIntCoder.of(),
+ KvCoder.of(windowCoder, VarLongCoder.of())));
+ PCollection<IsmRecord<WindowedValue<V>>> windowMapSizeMetadata = outputForSize
+ .apply("GBKaSVForSize", new GroupByKeyAndSortValuesOnly<Integer, W, Long>())
+ .apply(ParDo.of(new ToIsmMetadataRecordForSizeDoFn<K, V, W>(windowCoder)));
+ windowMapSizeMetadata.setCoder(ismCoder);
+
+ // Set the coder on the metadata output destined to build the entry set and process the
+ // entries producing a [META, Window, Index] record per window key pair storing the key.
+ PCollection<KV<Integer, KV<W, K>>> outputForEntrySet =
+ outputTuple.get(outputForEntrySetTag);
+ outputForEntrySet.setCoder(
+ KvCoder.of(VarIntCoder.of(),
+ KvCoder.of(windowCoder, inputCoder.getKeyCoder())));
+ PCollection<IsmRecord<WindowedValue<V>>> windowMapKeysMetadata = outputForEntrySet
+ .apply("GBKaSVForKeys", new GroupByKeyAndSortValuesOnly<Integer, W, K>())
+ .apply(ParDo.of(
+ new ToIsmMetadataRecordForKeyDoFn<K, V, W>(inputCoder.getKeyCoder(), windowCoder)));
+ windowMapKeysMetadata.setCoder(ismCoder);
+
+ // Set that all these outputs should be materialized using an indexed format.
+ runner.addPCollectionRequiringIndexedFormat(perHashWithReifiedWindows);
+ runner.addPCollectionRequiringIndexedFormat(windowMapSizeMetadata);
+ runner.addPCollectionRequiringIndexedFormat(windowMapKeysMetadata);
+
+ PCollectionList<IsmRecord<WindowedValue<V>>> outputs =
+ PCollectionList.of(ImmutableList.of(
+ perHashWithReifiedWindows, windowMapSizeMetadata, windowMapKeysMetadata));
+
+ return Pipeline.applyTransform(outputs,
+ Flatten.<IsmRecord<WindowedValue<V>>>pCollections())
+ .apply(CreatePCollectionView.<IsmRecord<WindowedValue<V>>,
+ ViewT>of(view));
+ }
+
+ @Override
+ protected String getKindString() {
+ return "BatchViewAsMultimap";
+ }
+
+ static <V> IsmRecordCoder<WindowedValue<V>> coderForMapLike(
+ Coder<? extends BoundedWindow> windowCoder, Coder<?> keyCoder, Coder<V> valueCoder) {
+ // TODO: swap to use a variable length long coder which has values which compare
+ // the same as their byte representation compare lexicographically within the key coder
+ return IsmRecordCoder.of(
+ 1, // We use only the key for hashing when producing value records
+ 2, // Since the key is not present, we add the window to the hash when
+ // producing metadata records
+ ImmutableList.of(
+ MetadataKeyCoder.of(keyCoder),
+ windowCoder,
+ BigEndianLongCoder.of()),
+ FullWindowedValueCoder.of(valueCoder, windowCoder));
+ }
+ }
+
+ /**
+ * A {@code Map<K, V2>} backed by a {@code Map<K, V1>} and a function that transforms
+ * {@code V1 -> V2}.
+ */
+ static class TransformedMap<K, V1, V2>
+ extends ForwardingMap<K, V2> {
+ private final Function<V1, V2> transform;
+ private final Map<K, V1> originalMap;
+ private final Map<K, V2> transformedMap;
+
+ private TransformedMap(Function<V1, V2> transform, Map<K, V1> originalMap) {
+ this.transform = transform;
+ this.originalMap = Collections.unmodifiableMap(originalMap);
+ this.transformedMap = Maps.transformValues(originalMap, transform);
+ }
+
+ @Override
+ protected Map<K, V2> delegate() {
+ return transformedMap;
+ }
+ }
+
+ /**
+ * A {@link Coder} for {@link TransformedMap}s.
+ */
+ static class TransformedMapCoder<K, V1, V2>
+ extends StandardCoder<TransformedMap<K, V1, V2>> {
+ private final Coder<Function<V1, V2>> transformCoder;
+ private final Coder<Map<K, V1>> originalMapCoder;
+
+ private TransformedMapCoder(
+ Coder<Function<V1, V2>> transformCoder, Coder<Map<K, V1>> originalMapCoder) {
+ this.transformCoder = transformCoder;
+ this.originalMapCoder = originalMapCoder;
+ }
+
+ public static <K, V1, V2> TransformedMapCoder<K, V1, V2> of(
+ Coder<Function<V1, V2>> transformCoder, Coder<Map<K, V1>> originalMapCoder) {
+ return new TransformedMapCoder<>(transformCoder, originalMapCoder);
+ }
+
+ @JsonCreator
+ public static <K, V1, V2> TransformedMapCoder<K, V1, V2> of(
+ @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
+ List<Coder<?>> components) {
+ checkArgument(components.size() == 2,
+ "Expecting 2 components, got " + components.size());
+ @SuppressWarnings("unchecked")
+ Coder<Function<V1, V2>> transformCoder = (Coder<Function<V1, V2>>) components.get(0);
+ @SuppressWarnings("unchecked")
+ Coder<Map<K, V1>> originalMapCoder = (Coder<Map<K, V1>>) components.get(1);
+ return of(transformCoder, originalMapCoder);
+ }
+
+ @Override
+ public void encode(TransformedMap<K, V1, V2> value, OutputStream outStream,
+ Coder.Context context) throws CoderException, IOException {
+ transformCoder.encode(value.transform, outStream, context.nested());
+ originalMapCoder.encode(value.originalMap, outStream, context.nested());
+ }
+
+ @Override
+ public TransformedMap<K, V1, V2> decode(
+ InputStream inStream, Coder.Contex
<TRUNCATED>