From de0fdbc1ee7f233e9ae363c7162171135d3b7403 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Sat, 29 Apr 2017 18:51:21 -0700 Subject: [PATCH 001/159] update JavaDoc for BoundedWindow update JavaDoc rephrase JavaDoc. --- .../sdk/transforms/windowing/BoundedWindow.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java index 74223b586715..506753dc4f77 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java @@ -21,11 +21,15 @@ import org.joda.time.Instant; /** - * A {@code BoundedWindow} represents a finite grouping of elements, with an - * upper bound (larger timestamps represent more recent data) on the timestamps - * of elements that can be placed in the window. This finiteness means that for - * every window, at some point in time, all data for that window will have - * arrived and can be processed together. + * A {@code BoundedWindow} represents window information assigned to data elements. + * + *

It has one method {@link #maxTimestamp()} to define an upper bound(included) of element + * timestamp. Elements within the timestamp range are placed in this window. + * Technically a window may not have a lower bound. Upper bound is mandatory as watermark, + * which also controls discarding data and expiring the window, is based on event timestamp. + * + *

At some point in time, usually when watermark passes, + * all data with the same window will arrive and can be processed together. * *

Windows must also implement {@link Object#equals} and * {@link Object#hashCode} such that windows that are logically equal will From 9b5167483c89423a0789658a33e51b3f5b6e8e16 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 18 May 2017 08:32:29 -0700 Subject: [PATCH 002/159] Restore non-recursive globs in GcsUtil --- .../java/org/apache/beam/sdk/util/GcsUtil.java | 8 +++++++- .../org/apache/beam/sdk/util/GcsUtilTest.java | 16 +++++++++++----- 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java index 94b733a7f8dc..8d1fe74ad270 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java @@ -176,7 +176,13 @@ public static String wildcardToRegexp(String globExp) { char c = src[i++]; switch (c) { case '*': - dst.append(".*"); + // One char lookahead for ** + if (i < src.length && src[i] == '*') { + dst.append(".*"); + ++i; + } else { + dst.append("[^/]*"); + } break; case '?': dst.append("[^/]"); diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java index 532645047d5a..2164e75c9cfe 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsUtilTest.java @@ -93,11 +93,17 @@ public class GcsUtilTest { @Test public void testGlobTranslation() { assertEquals("foo", GcsUtil.wildcardToRegexp("foo")); - assertEquals("fo.*o", GcsUtil.wildcardToRegexp("fo*o")); - assertEquals("f.*o\\.[^/]", GcsUtil.wildcardToRegexp("f*o.?")); - assertEquals("foo-[0-9].*", GcsUtil.wildcardToRegexp("foo-[0-9]*")); - assertEquals(".*.*foo", GcsUtil.wildcardToRegexp("**/*foo")); - assertEquals(".*.*foo", GcsUtil.wildcardToRegexp("**foo")); + assertEquals("fo[^/]*o", GcsUtil.wildcardToRegexp("fo*o")); + assertEquals("f[^/]*o\\.[^/]", GcsUtil.wildcardToRegexp("f*o.?")); + assertEquals("foo-[0-9][^/]*", GcsUtil.wildcardToRegexp("foo-[0-9]*")); + assertEquals("foo-[0-9].*", GcsUtil.wildcardToRegexp("foo-[0-9]**")); + assertEquals(".*foo", GcsUtil.wildcardToRegexp("**/*foo")); + assertEquals(".*foo", GcsUtil.wildcardToRegexp("**foo")); + assertEquals("foo/[^/]*", GcsUtil.wildcardToRegexp("foo/*")); + assertEquals("foo[^/]*", GcsUtil.wildcardToRegexp("foo*")); + assertEquals("foo/[^/]*/[^/]*/[^/]*", GcsUtil.wildcardToRegexp("foo/*/*/*")); + assertEquals("foo/[^/]*/.*", GcsUtil.wildcardToRegexp("foo/*/**")); + assertEquals("foo.*baz", GcsUtil.wildcardToRegexp("foo**baz")); } private static GcsOptions gcsOptionsWithTestCredential() { From 66e564e507b1ad5b9e379e043a193b556e1f3e76 Mon Sep 17 00:00:00 2001 From: Borisa Zivkovic Date: Mon, 15 May 2017 09:24:10 +0100 Subject: [PATCH 003/159] [BEAM-1702] Document how to use BigtableIO with local emulator --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 26 +++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 22e9f3611ec7..1692cda65623 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -139,6 +139,30 @@ * .withTableId("table")); * } * + *

Using local emulator

+ * + *

In order to use local emulator for Bigtable you should use: + * + *

{@code
+ * BigtableOptions.Builder optionsBuilder =
+ *     new BigtableOptions.Builder()
+ *         .setProjectId("project")
+ *         .setInstanceId("instance")
+ *         .setUsePlaintextNegotiation(true)
+ *         .setCredentialOptions(CredentialOptions.nullCredential())
+ *         .setDataHost("127.0.0.1") // network interface where Bigtable emulator is bound
+ *         .setInstanceAdminHost("127.0.0.1")
+ *         .setTableAdminHost("127.0.0.1")
+ *         .setPort(LOCAL_EMULATOR_PORT))
+ *
+ * PCollection>> data = ...;
+ *
+ * data.apply("write",
+ *     BigtableIO.write()
+ *         .withBigtableOptions(optionsBuilder)
+ *         .withTableId("table");
+ * }
+ * *

Experimental

* *

This connector for Cloud Bigtable is considered experimental and may break or receive @@ -205,7 +229,6 @@ public abstract static class Read extends PTransform> { @Nullable abstract BigtableService getBigtableService(); - /** Returns the Google Cloud Bigtable instance being read from, and other parameters. */ @Nullable public abstract BigtableOptions getBigtableOptions(); @@ -429,7 +452,6 @@ abstract static class Builder { *

Does not modify this object. */ public Write withBigtableOptions(BigtableOptions options) { - checkNotNull(options, "options"); return withBigtableOptions(options.toBuilder()); } From a8d2125e5783a556056e88dad8fe3c0a397920d5 Mon Sep 17 00:00:00 2001 From: huafengw Date: Tue, 9 May 2017 11:21:44 +0800 Subject: [PATCH 004/159] [BEAM-659] WindowFn#isCompatible should provide a meaningful reason --- .../core/construction/PCollectionsTest.java | 12 ++++++ .../direct/WindowEvaluatorFactoryTest.java | 10 +++++ .../beam/sdk/testing/StaticWindows.java | 12 ++++++ .../transforms/windowing/CalendarWindows.java | 36 ++++++++++++++++++ .../transforms/windowing/FixedWindows.java | 11 ++++++ .../transforms/windowing/GlobalWindows.java | 11 ++++++ .../IncompatibleWindowException.java | 38 +++++++++++++++++++ .../transforms/windowing/InvalidWindows.java | 11 ++++++ .../sdk/transforms/windowing/Sessions.java | 11 ++++++ .../transforms/windowing/SlidingWindows.java | 11 ++++++ .../sdk/transforms/windowing/WindowFn.java | 21 ++++++++++ .../beam/sdk/util/IdentityWindowFn.java | 11 ++++++ .../beam/sdk/testing/StaticWindowsTest.java | 12 ++++++ .../windowing/CalendarWindowsTest.java | 24 ++++++++++++ .../windowing/FixedWindowsTest.java | 7 ++++ .../transforms/windowing/SessionsTest.java | 14 +++++++ .../windowing/SlidingWindowsTest.java | 11 ++++++ .../sdk/transforms/windowing/WindowTest.java | 8 ++++ .../sdk/util/IdentitySideInputWindowFn.java | 4 ++ .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 13 +++++++ 20 files changed, 288 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IncompatibleWindowException.java diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java index 2c45cbdb94b3..66700d0d6b30 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java @@ -44,6 +44,7 @@ import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException; import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; @@ -157,6 +158,17 @@ public boolean isCompatible(WindowFn other) { return other != null && this.getClass().equals(other.getClass()); } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "%s is only compatible with %s.", + CustomWindows.class.getSimpleName(), CustomWindows.class.getSimpleName())); + } + } + @Override public Coder windowCoder() { return new AtomicCoder() { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java index a91bab54fae8..96fdfaba8b9f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -303,6 +304,15 @@ public boolean isCompatible(WindowFn other) { return false; } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + throw new IncompatibleWindowException( + other, + String.format( + "%s is not compatible with any other %s.", + EvaluatorTestWindowFn.class.getSimpleName(), WindowFn.class.getSimpleName())); + } + @Override public Coder windowCoder() { @SuppressWarnings({"unchecked", "rawtypes"}) Coder coder = diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java index fde16695808e..c11057a5c623 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java @@ -26,6 +26,7 @@ import java.util.Objects; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException; import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; @@ -96,6 +97,17 @@ public boolean isCompatible(WindowFn other) { return Objects.equals(this.windows.get(), that.windows.get()); } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "Only %s objects with the same window supplier are compatible.", + StaticWindows.class.getSimpleName())); + } + } + @Override public Coder windowCoder() { return coder; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java index fada50ad5a4d..989c43133220 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java @@ -144,6 +144,18 @@ public boolean isCompatible(WindowFn other) { && timeZone == that.timeZone; } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "Only %s objects with the same number of days, start date " + + "and time zone are compatible.", + DaysWindows.class.getSimpleName())); + } + } + @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); @@ -244,6 +256,18 @@ public boolean isCompatible(WindowFn other) { && timeZone == that.timeZone; } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "Only %s objects with the same number of months, " + + "day of month, start date and time zone are compatible.", + MonthsWindows.class.getSimpleName())); + } + } + @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); @@ -353,6 +377,18 @@ public boolean isCompatible(WindowFn other) { && timeZone == that.timeZone; } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "Only %s objects with the same number of years, month of year, " + + "day of month, start date and time zone are compatible.", + YearsWindows.class.getSimpleName())); + } + } + @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java index 8683a600e204..8b169169a780 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java @@ -101,6 +101,17 @@ public boolean isCompatible(WindowFn other) { return this.equals(other); } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "Only %s objects with the same size and offset are compatible.", + FixedWindows.class.getSimpleName())); + } + } + public Duration getSize() { return size; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java index 400be1fe7095..b49328b20198 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java @@ -40,6 +40,17 @@ public boolean isCompatible(WindowFn o) { return o instanceof GlobalWindows; } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "%s is only compatible with %s.", + GlobalWindows.class.getSimpleName(), GlobalWindows.class.getSimpleName())); + } + } + @Override public Coder windowCoder() { return GlobalWindow.Coder.INSTANCE; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IncompatibleWindowException.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IncompatibleWindowException.java new file mode 100644 index 000000000000..b7b96add2293 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IncompatibleWindowException.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms.windowing; + +/** + * Exception thrown by {@link WindowFn#verifyCompatibility(WindowFn)} if two compared + * WindowFns are not compatible, including the explanation of incompatibility. + */ +public class IncompatibleWindowException extends Exception { + private WindowFn givenWindowFn; + private String reason; + + public IncompatibleWindowException(WindowFn windowFn, String reason) { + this.givenWindowFn = windowFn; + this.reason = reason; + } + + @Override + public String getMessage() { + String windowFn = givenWindowFn == null ? "null" : givenWindowFn.getClass().getSimpleName(); + return String.format("The given WindowFn is %s. %s", windowFn, reason); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java index 92041fc54c73..a8084f42d993 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java @@ -74,6 +74,17 @@ && getOriginalWindowFn().isCompatible( ((InvalidWindows) other).getOriginalWindowFn()); } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "Only %s objects with the same originalWindowFn are compatible.", + InvalidWindows.class.getSimpleName())); + } + } + @Override public WindowMappingFn getDefaultWindowMappingFn() { throw new UnsupportedOperationException("InvalidWindows is not allowed in side inputs"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java index 5cc7c65b7da5..115a964740b4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java @@ -79,6 +79,17 @@ public boolean isCompatible(WindowFn other) { return other instanceof Sessions; } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "%s is only compatible with %s.", + Sessions.class.getSimpleName(), Sessions.class.getSimpleName())); + } + } + @Override public WindowMappingFn getDefaultWindowMappingFn() { throw new UnsupportedOperationException("Sessions is not allowed in side inputs"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java index 650dc37232b0..f65788438021 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java @@ -147,6 +147,17 @@ public boolean isCompatible(WindowFn other) { return equals(other); } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "Only %s objects with the same size, period and offset are compatible.", + SlidingWindows.class.getSimpleName())); + } + } + @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java index 5ebbb41b5956..e329c1d1ace8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java @@ -114,9 +114,30 @@ public abstract void merge(Collection toBeMerged, W mergeResult) /** * Returns whether this performs the same merging as the given * {@code WindowFn}. + * + * @deprecated please override verifyCompatibility to throw a useful error message; + * we will remove isCompatible at version 3.0.0 */ + @Deprecated public abstract boolean isCompatible(WindowFn other); + /** + * Throw {@link IncompatibleWindowException} if this WindowFn does not perform the same merging as + * the given ${@code WindowFn}. + * + * @throws IncompatibleWindowException if compared WindowFns are not compatible. + */ + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "%s is not compatible with %s", + this.getClass().getSimpleName(), + other.getClass().getSimpleName())); + } + } + /** * Returns the {@link Coder} used for serializing the windows used * by this windowFn. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java index a61e3a6482dc..a4bfdda4f813 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java @@ -23,6 +23,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException; import org.apache.beam.sdk.transforms.windowing.InvalidWindows; import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.Window; @@ -83,6 +84,16 @@ public boolean isCompatible(WindowFn other) { getClass().getCanonicalName())); } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + throw new UnsupportedOperationException( + String.format( + "%s.verifyCompatibility() should never be called." + + " It is a private implementation detail of sdk utilities." + + " This message indicates a bug in the Beam SDK.", + getClass().getCanonicalName())); + } + @Override public Coder windowCoder() { // Safe because the prior WindowFn provides both the windows and the coder. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java index 7ee48c8c1ecb..2969ca66b362 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.hamcrest.Matchers; @@ -93,4 +94,15 @@ public void emptyIterableThrows() { thrown.expectMessage("may not be empty"); StaticWindows.of(GlobalWindow.Coder.INSTANCE, ImmutableList.of()); } + + @Test + public void testCompatibility() throws IncompatibleWindowException { + StaticWindows staticWindows = + StaticWindows.of(IntervalWindow.getCoder(), ImmutableList.of(first, second)); + staticWindows.verifyCompatibility( + StaticWindows.of(IntervalWindow.getCoder(), ImmutableList.of(first, second))); + thrown.expect(IncompatibleWindowException.class); + staticWindows.verifyCompatibility( + StaticWindows.of(IntervalWindow.getCoder(), ImmutableList.of(first))); + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java index cd562e9bea17..c8c01f5e4882 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java @@ -90,6 +90,14 @@ public void testDays() throws Exception { assertEquals(expected, runWindowFn(CalendarWindows.days(1), timestamps)); } + @Test + public void testDaysCompatibility() throws IncompatibleWindowException { + CalendarWindows.DaysWindows daysWindows = CalendarWindows.days(10); + daysWindows.verifyCompatibility(CalendarWindows.days(10)); + thrown.expect(IncompatibleWindowException.class); + daysWindows.verifyCompatibility(CalendarWindows.days(9)); + } + @Test public void testWeeks() throws Exception { Map> expected = new HashMap<>(); @@ -164,6 +172,14 @@ public void testMonths() throws Exception { runWindowFn(CalendarWindows.months(1), timestamps)); } + @Test + public void testMonthsCompatibility() throws IncompatibleWindowException { + CalendarWindows.MonthsWindows monthsWindows = CalendarWindows.months(10).beginningOnDay(15); + monthsWindows.verifyCompatibility(CalendarWindows.months(10).beginningOnDay(15)); + thrown.expect(IncompatibleWindowException.class); + monthsWindows.verifyCompatibility(CalendarWindows.months(10).beginningOnDay(30)); + } + @Test public void testMultiMonths() throws Exception { Map> expected = new HashMap<>(); @@ -238,6 +254,14 @@ public void testYears() throws Exception { CalendarWindows.years(10).withStartingYear(2000).beginningOnDay(5, 5), timestamps)); } + @Test + public void testYearsCompatibility() throws IncompatibleWindowException { + CalendarWindows.YearsWindows yearsWindows = CalendarWindows.years(2017).beginningOnDay(1, 1); + yearsWindows.verifyCompatibility(CalendarWindows.years(2017).beginningOnDay(1, 1)); + thrown.expect(IncompatibleWindowException.class); + yearsWindows.verifyCompatibility(CalendarWindows.years(2017).beginningOnDay(1, 2)); + } + @Test public void testTimeZone() throws Exception { Map> expected = new HashMap<>(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java index 47c273a13c54..80a534c5550a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java @@ -148,6 +148,13 @@ public void testEquality() { FixedWindows.of(new Duration(20)))); } + @Test + public void testVerifyCompatibility() throws IncompatibleWindowException { + FixedWindows.of(new Duration(10)).verifyCompatibility(FixedWindows.of(new Duration(10))); + thrown.expect(IncompatibleWindowException.class); + FixedWindows.of(new Duration(10)).verifyCompatibility(FixedWindows.of(new Duration(20))); + } + @Test public void testValidOutputTimes() throws Exception { for (long timestamp : Arrays.asList(200, 800, 700)) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java index 9d949286e02e..42c15b5233a3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java @@ -36,7 +36,9 @@ import org.apache.beam.sdk.transforms.display.DisplayData; 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; @@ -45,6 +47,8 @@ */ @RunWith(JUnit4.class) public class SessionsTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); @Test public void testSimple() throws Exception { @@ -106,6 +110,16 @@ public void testEquality() { Sessions.withGapDuration(new Duration(20)))); } + @Test + public void testVerifyCompatibility() throws IncompatibleWindowException { + Sessions.withGapDuration(new Duration(10)) + .verifyCompatibility(Sessions.withGapDuration(new Duration(10))); + + thrown.expect(IncompatibleWindowException.class); + Sessions.withGapDuration(new Duration(10)) + .verifyCompatibility(FixedWindows.of(new Duration(10))); + } + /** * Validates that the output timestamp for aggregate data falls within the acceptable range. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java index dd673d318b20..b14e2215173d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java @@ -34,7 +34,9 @@ import org.apache.beam.sdk.transforms.display.DisplayData; 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; @@ -43,6 +45,8 @@ */ @RunWith(JUnit4.class) public class SlidingWindowsTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); @Test public void testSimple() throws Exception { @@ -152,6 +156,13 @@ public void testEquality() { SlidingWindows.of(new Duration(20)))); } + @Test + public void testVerifyCompatibility() throws IncompatibleWindowException { + SlidingWindows.of(new Duration(10)).verifyCompatibility(SlidingWindows.of(new Duration(10))); + thrown.expect(IncompatibleWindowException.class); + SlidingWindows.of(new Duration(10)).verifyCompatibility(SlidingWindows.of(new Duration(20))); + } + @Test public void testDefaultWindowMappingFn() { // [40, 1040), [340, 1340), [640, 1640) ... diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java index 92f6a9c01457..f536a9a9b8a1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java @@ -300,6 +300,14 @@ public boolean isCompatible(WindowFn other) { return other instanceof WindowOddEvenBuckets; } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, "WindowOddEvenBuckets is only compatible with WindowOddEvenBuckets."); + } + } + @Override public Coder windowCoder() { return new IntervalWindow.IntervalWindowCoder(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java index 2171466b1192..32e23dab142f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java @@ -22,6 +22,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException; import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; @@ -42,6 +43,9 @@ public boolean isCompatible(WindowFn other) { return true; } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException {} + @Override public Coder windowCoder() { // not used diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index d60c7214b701..ba0cea81127c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -115,6 +115,7 @@ import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.Window; @@ -697,6 +698,18 @@ public boolean isCompatible(WindowFn o) { return o instanceof PartitionedGlobalWindows; } + @Override + public void verifyCompatibility(WindowFn other) throws IncompatibleWindowException { + if (!this.isCompatible(other)) { + throw new IncompatibleWindowException( + other, + String.format( + "%s is only compatible with %s.", + PartitionedGlobalWindows.class.getSimpleName(), + PartitionedGlobalWindows.class.getSimpleName())); + } + } + @Override public Coder windowCoder() { return new PartitionedGlobalWindowCoder(); From 26a615e885b39f92dc81d383c8115906c280187a Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Thu, 18 May 2017 15:52:09 -0400 Subject: [PATCH 005/159] Remove incorrect `Message` bound on writeAvros --- .../java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index e293b95ede1e..4f33d61ae403 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -527,7 +527,7 @@ public static Write writeProtos(Class messageClass) { * Returns A {@link PTransform} that writes binary encoded Avro messages of a given type * to a Google Cloud Pub/Sub stream. */ - public static Write writeAvros(Class clazz) { + public static Write writeAvros(Class clazz) { // TODO: Like in readAvros(), stop using AvroCoder and instead format the payload directly. return PubsubIO.write().withFormatFn(new FormatPayloadUsingCoder<>(AvroCoder.of(clazz))); } @@ -970,8 +970,7 @@ public PubsubMessage apply(String input) { } } - private static class FormatPayloadUsingCoder - extends SimpleFunction { + private static class FormatPayloadUsingCoder extends SimpleFunction { private Coder coder; public FormatPayloadUsingCoder(Coder coder) { From 790e7fe6653b926044d3dfecdccbc2fda9c998f0 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 21 Mar 2017 15:06:58 -0700 Subject: [PATCH 006/159] Add ParDos Add ParDoPayloadTranslator to PTransformTranslator --- .../core/construction/PTransforms.java | 8 +- .../runners/core/construction/ParDos.java | 317 ++++++++++++++++++ .../construction/RunnerPCollectionView.java | 88 +++++ .../runners/core/construction/ParDosTest.java | 229 +++++++++++++ .../src/main/proto/beam_runner_api.proto | 10 + .../transforms/windowing/GlobalWindows.java | 6 +- 6 files changed, 653 insertions(+), 5 deletions(-) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java index d25d342bfd65..16276b9ed82c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java @@ -24,10 +24,12 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.construction.ParDos.ParDoPayloadTranslator; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; @@ -39,9 +41,9 @@ public class PTransforms { private static final Map, TransformPayloadTranslator> KNOWN_PAYLOAD_TRANSLATORS = - ImmutableMap., TransformPayloadTranslator>builder().build(); - // TODO: ParDoPayload, WindowIntoPayload, ReadPayload, CombinePayload - // TODO: "Flatten Payload", etc? + ImmutableMap., TransformPayloadTranslator>builder() + .put(ParDo.MultiOutput.class, ParDoPayloadTranslator.create()) + .build(); // TODO: Load via service loader. private PTransforms() {} diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java new file mode 100644 index 000000000000..b2b29df65dc0 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import com.google.common.base.Optional; +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; +import com.google.protobuf.InvalidProtocolBufferException; +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import org.apache.beam.runners.core.construction.PTransforms.TransformPayloadTranslator; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.Components; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.ParDoPayload; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.Parameter.Type; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SideInput; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SideInput.Builder; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.StateSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.TimerSpec; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Materializations; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.ParDo.MultiOutput; +import org.apache.beam.sdk.transforms.ViewFn; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.Cases; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.RestrictionTrackerParameter; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.WindowParameter; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature.StateDeclaration; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature.TimerDeclaration; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; + +/** + * Utilities for interacting with {@link ParDo} instances and {@link ParDoPayload} protos. + */ +public class ParDos { + /** + * The URN for a {@link ParDoPayload}. + */ + public static final String PAR_DO_PAYLOAD_URN = "urn:beam:pardo:v1"; + /** + * The URN for an unknown Java {@link DoFn}. + */ + public static final String CUSTOM_JAVA_DO_FN_URN = "urn:beam:dofn:javasdk:0.1"; + /** + * The URN for an unknown Java {@link ViewFn}. + */ + public static final String CUSTOM_JAVA_VIEW_FN_URN = "urn:beam:viewfn:javasdk:0.1"; + /** + * The URN for an unknown Java {@link WindowMappingFn}. + */ + public static final String CUSTOM_JAVA_WINDOW_MAPPING_FN_URN = + "urn:beam:windowmappingfn:javasdk:0.1"; + + /** + * A {@link TransformPayloadTranslator} for {@link ParDo}. + */ + public static class ParDoPayloadTranslator + implements PTransforms.TransformPayloadTranslator> { + public static TransformPayloadTranslator create() { + return new ParDoPayloadTranslator(); + } + + private ParDoPayloadTranslator() {} + + @Override + public FunctionSpec translate( + AppliedPTransform> transform, SdkComponents components) { + ParDoPayload payload = toProto(transform.getTransform(), components); + return RunnerApi.FunctionSpec.newBuilder() + .setUrn(PAR_DO_PAYLOAD_URN) + .setParameter(Any.pack(payload)) + .build(); + } + } + + public static ParDoPayload toProto(ParDo.MultiOutput parDo, SdkComponents components) { + DoFnSignature signature = DoFnSignatures.getSignature(parDo.getFn().getClass()); + Map states = signature.stateDeclarations(); + Map timers = signature.timerDeclarations(); + List parameters = signature.processElement().extraParameters(); + + ParDoPayload.Builder builder = ParDoPayload.newBuilder(); + builder.setDoFn(toProto(parDo.getFn(), parDo.getMainOutputTag())); + for (PCollectionView sideInput : parDo.getSideInputs()) { + builder.putSideInputs(sideInput.getTagInternal().getId(), toProto(sideInput)); + } + for (Parameter parameter : parameters) { + Optional protoParameter = toProto(parameter); + if (protoParameter.isPresent()) { + builder.addParameters(protoParameter.get()); + } + } + for (Map.Entry state : states.entrySet()) { + StateSpec spec = toProto(state.getValue()); + builder.putStateSpecs(state.getKey(), spec); + } + for (Map.Entry timer : timers.entrySet()) { + TimerSpec spec = toProto(timer.getValue()); + builder.putTimerSpecs(timer.getKey(), spec); + } + return builder.build(); + } + + public static DoFn getDoFn(ParDoPayload payload) throws InvalidProtocolBufferException { + return doFnAndMainOutputTagFromProto(payload.getDoFn()).getDoFn(); + } + + public static TupleTag getMainOutputTag(ParDoPayload payload) + throws InvalidProtocolBufferException { + return doFnAndMainOutputTagFromProto(payload.getDoFn()).getMainOutputTag(); + } + + // TODO: Implement + private static StateSpec toProto(StateDeclaration state) { + throw new UnsupportedOperationException("Not yet supported"); + } + + // TODO: Implement + private static TimerSpec toProto(TimerDeclaration timer) { + throw new UnsupportedOperationException("Not yet supported"); + } + + @AutoValue + abstract static class DoFnAndMainOutput implements Serializable { + public static DoFnAndMainOutput of( + DoFn fn, TupleTag tag) { + return new AutoValue_ParDos_DoFnAndMainOutput(fn, tag); + } + + abstract DoFn getDoFn(); + abstract TupleTag getMainOutputTag(); + } + + private static SdkFunctionSpec toProto(DoFn fn, TupleTag tag) { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(CUSTOM_JAVA_DO_FN_URN) + .setParameter( + Any.pack( + BytesValue.newBuilder() + .setValue( + ByteString.copyFrom( + SerializableUtils.serializeToByteArray( + DoFnAndMainOutput.of(fn, tag)))) + .build()))) + .build(); + } + + private static DoFnAndMainOutput doFnAndMainOutputTagFromProto(SdkFunctionSpec fnSpec) + throws InvalidProtocolBufferException { + checkArgument(fnSpec.getSpec().getUrn().equals(CUSTOM_JAVA_DO_FN_URN)); + byte[] serializedFn = + fnSpec.getSpec().getParameter().unpack(BytesValue.class).getValue().toByteArray(); + return (DoFnAndMainOutput) + SerializableUtils.deserializeFromByteArray(serializedFn, "Custom DoFn And Main Output tag"); + } + + private static Optional toProto(Parameter parameter) { + return parameter.match( + new Cases.WithDefault>() { + @Override + public Optional dispatch(WindowParameter p) { + return Optional.of(RunnerApi.Parameter.newBuilder().setType(Type.WINDOW).build()); + } + + @Override + public Optional dispatch(RestrictionTrackerParameter p) { + return Optional.of( + RunnerApi.Parameter.newBuilder().setType(Type.RESTRICTION_TRACKER).build()); + } + + @Override + protected Optional dispatchDefault(Parameter p) { + return Optional.absent(); + } + }); + } + + private static SideInput toProto(PCollectionView view) { + Builder builder = SideInput.newBuilder(); + builder.setAccessPattern( + FunctionSpec.newBuilder() + .setUrn(view.getViewFn().getMaterialization().getUrn()) + .build()); + builder.setViewFn(toProto(view.getViewFn())); + builder.setWindowMappingFn(toProto(view.getWindowMappingFn())); + return builder.build(); + } + + public static PCollectionView fromProto( + SideInput sideInput, String id, RunnerApi.PTransform parDoTransform, Components components) + throws IOException { + TupleTag tag = new TupleTag<>(id); + WindowMappingFn windowMappingFn = windowMappingFnFromProto(sideInput.getWindowMappingFn()); + ViewFn viewFn = viewFnFromProto(sideInput.getViewFn()); + + RunnerApi.PCollection inputCollection = + components.getPcollectionsOrThrow(parDoTransform.getInputsOrThrow(id)); + WindowingStrategy windowingStrategy = + WindowingStrategies.fromProto( + components.getWindowingStrategiesOrThrow(inputCollection.getWindowingStrategyId()), + components); + Coder elemCoder = + Coders.fromProto(components.getCodersOrThrow(inputCollection.getCoderId()), components); + Coder>> coder = + (Coder) + IterableCoder.of( + FullWindowedValueCoder.of( + elemCoder, windowingStrategy.getWindowFn().windowCoder())); + checkArgument( + sideInput.getAccessPattern().getUrn().equals(Materializations.ITERABLE_MATERIALIZATION_URN), + "Unknown View Materialization URN %s", + sideInput.getAccessPattern().getUrn()); + + PCollectionView view = + new RunnerPCollectionView<>( + (TupleTag>>) tag, + (ViewFn>, ?>) viewFn, + windowMappingFn, + windowingStrategy, + coder); + return view; + } + + private static SdkFunctionSpec toProto(ViewFn viewFn) { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(CUSTOM_JAVA_VIEW_FN_URN) + .setParameter( + Any.pack( + BytesValue.newBuilder() + .setValue( + ByteString.copyFrom(SerializableUtils.serializeToByteArray(viewFn))) + .build()))) + .build(); + } + + private static ViewFn viewFnFromProto(SdkFunctionSpec viewFn) + throws InvalidProtocolBufferException { + FunctionSpec spec = viewFn.getSpec(); + checkArgument( + spec.getUrn().equals(CUSTOM_JAVA_VIEW_FN_URN), + "Can't deserialize unknown %s type %s", + ViewFn.class.getSimpleName(), + spec.getUrn()); + return (ViewFn) + SerializableUtils.deserializeFromByteArray( + spec.getParameter().unpack(BytesValue.class).getValue().toByteArray(), "Custom ViewFn"); + } + + private static SdkFunctionSpec toProto(WindowMappingFn windowMappingFn) { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(CUSTOM_JAVA_WINDOW_MAPPING_FN_URN) + .setParameter( + Any.pack( + BytesValue.newBuilder() + .setValue( + ByteString.copyFrom( + SerializableUtils.serializeToByteArray(windowMappingFn))) + .build()))) + .build(); + } + + private static WindowMappingFn windowMappingFnFromProto(SdkFunctionSpec windowMappingFn) + throws InvalidProtocolBufferException { + FunctionSpec spec = windowMappingFn.getSpec(); + checkArgument( + spec.getUrn().equals(CUSTOM_JAVA_WINDOW_MAPPING_FN_URN), + "Can't deserialize unknown %s type %s", + WindowMappingFn.class.getSimpleName(), + spec.getUrn()); + return (WindowMappingFn) + SerializableUtils.deserializeFromByteArray( + spec.getParameter().unpack(BytesValue.class).getValue().toByteArray(), + "Custom WinodwMappingFn"); + } +} diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java new file mode 100644 index 000000000000..89e878496ef3 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SideInput; +import org.apache.beam.sdk.transforms.ViewFn; +import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PValueBase; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; + +/** A {@link PCollectionView} created from the components of a {@link SideInput}. */ +class RunnerPCollectionView extends PValueBase implements PCollectionView { + private final TupleTag>> tag; + private final ViewFn>, T> viewFn; + private final WindowMappingFn windowMappingFn; + private final WindowingStrategy windowingStrategy; + private final Coder>> coder; + + /** + * Create a new {@link RunnerPCollectionView} from the provided components. + */ + RunnerPCollectionView( + TupleTag>> tag, + ViewFn>, T> viewFn, + WindowMappingFn windowMappingFn, + @Nullable WindowingStrategy windowingStrategy, + @Nullable Coder>> coder) { + this.tag = tag; + this.viewFn = viewFn; + this.windowMappingFn = windowMappingFn; + this.windowingStrategy = windowingStrategy; + this.coder = coder; + } + + @Nullable + @Override + public PCollection getPCollection() { + throw new IllegalStateException( + String.format("Cannot call getPCollection on a %s", getClass().getSimpleName())); + } + + @Override + public TupleTag>> getTagInternal() { + return tag; + } + + @Override + public ViewFn>, T> getViewFn() { + return viewFn; + } + + @Override + public WindowMappingFn getWindowMappingFn() { + return windowMappingFn; + } + + @Override + public WindowingStrategy getWindowingStrategyInternal() { + return windowingStrategy; + } + + @Override + public Coder>> getCoderInternal() { + return coder; + } +} diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java new file mode 100644 index 000000000000..74edec15af81 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertThat; + +import com.google.common.collect.ImmutableList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.Components; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.ParDoPayload; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SideInput; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Combine.BinaryCombineLongFn; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.ParDo.MultiOutput; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +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.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.hamcrest.Matchers; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; + +/** Tests for {@link ParDos}. */ +@RunWith(Parameterized.class) +public class ParDosTest { + public static TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + + private static PCollectionView singletonSideInput = + p.apply("GenerateSingleton", GenerateSequence.from(0L).to(1L)) + .apply(View.asSingleton()); + private static PCollectionView>> multimapSideInput = + p.apply("CreateMultimap", Create.of(KV.of(1L, "foo"), KV.of(1L, "bar"), KV.of(2L, "spam"))) + .setCoder(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of())) + .apply(View.asMultimap()); + + private static PCollection> mainInput = + p.apply("CreateMainInput", Create.empty(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of()))); + + @Parameters(name = "{index}: {0}") + public static Iterable> data() { + return ImmutableList.>of( + ParDo.of(new DropElementsFn()).withOutputTags(new TupleTag(), TupleTagList.empty()), + ParDo.of(new DropElementsFn()) + .withOutputTags(new TupleTag(), TupleTagList.empty()) + .withSideInputs(singletonSideInput, multimapSideInput), + ParDo.of(new DropElementsFn()) + .withOutputTags( + new TupleTag(), + TupleTagList.of(new TupleTag() {}).and(new TupleTag() {})) + .withSideInputs(singletonSideInput, multimapSideInput), + ParDo.of(new DropElementsFn()) + .withOutputTags( + new TupleTag(), + TupleTagList.of(new TupleTag() {}).and(new TupleTag() {}))); + } + + @Parameter(0) + public ParDo.MultiOutput, Void> parDo; + + @Test + public void testToAndFromProto() throws Exception { + SdkComponents components = SdkComponents.create(); + ParDoPayload payload = ParDos.toProto(parDo, components); + + assertThat(ParDos.getDoFn(payload), Matchers.>equalTo(parDo.getFn())); + assertThat( + ParDos.getMainOutputTag(payload), Matchers.>equalTo(parDo.getMainOutputTag())); + for (PCollectionView view : parDo.getSideInputs()) { + payload.getSideInputsOrThrow(view.getTagInternal().getId()); + } + } + + @Test + public void toAndFromTransformProto() throws Exception { + Map, PValue> inputs = new HashMap<>(); + inputs.put(new TupleTag>() {}, mainInput); + inputs.putAll(parDo.getAdditionalInputs()); + PCollectionTuple output = mainInput.apply(parDo); + + SdkComponents components = SdkComponents.create(); + String transformId = + components.registerPTransform( + AppliedPTransform.>, PCollection, MultiOutput>of( + "foo", inputs, output.expand(), parDo, p), + Collections.>emptyList()); + + Components protoComponents = components.toComponents(); + RunnerApi.PTransform protoTransform = + protoComponents.getTransformsOrThrow(transformId); + ParDoPayload parDoPayload = protoTransform.getSpec().getParameter().unpack(ParDoPayload.class); + for (PCollectionView view : parDo.getSideInputs()) { + SideInput sideInput = parDoPayload.getSideInputsOrThrow(view.getTagInternal().getId()); + PCollectionView restoredView = + ParDos.fromProto( + sideInput, view.getTagInternal().getId(), protoTransform, protoComponents); + assertThat(restoredView.getTagInternal(), equalTo(view.getTagInternal())); + assertThat(restoredView.getViewFn(), instanceOf(view.getViewFn().getClass())); + assertThat( + restoredView.getWindowMappingFn(), instanceOf(view.getWindowMappingFn().getClass())); + assertThat( + restoredView.getWindowingStrategyInternal(), + Matchers.>equalTo( + view.getWindowingStrategyInternal().fixDefaults())); + assertThat(restoredView.getCoderInternal(), equalTo(view.getCoderInternal())); + } + } + + private static class DropElementsFn extends DoFn, Void> { + @ProcessElement + public void proc(ProcessContext context, BoundedWindow window) { + context.output(null); + } + + @Override + public boolean equals(Object other) { + return other instanceof DropElementsFn; + } + + @Override + public int hashCode() { + return DropElementsFn.class.hashCode(); + } + } + + @SuppressWarnings("unused") + private static class StateTimerDropElementsFn extends DoFn, Void> { + private static final String BAG_STATE_ID = "bagState"; + private static final String COMBINING_STATE_ID = "combiningState"; + private static final String EVENT_TIMER_ID = "eventTimer"; + private static final String PROCESSING_TIMER_ID = "processingTimer"; + + @StateId(BAG_STATE_ID) + private final StateSpec> bagState = StateSpecs.bag(StringUtf8Coder.of()); + + @StateId(COMBINING_STATE_ID) + private final StateSpec> combiningState = + StateSpecs.combining( + new BinaryCombineLongFn() { + @Override + public long apply(long left, long right) { + return Math.max(left, right); + } + + @Override + public long identity() { + return Long.MIN_VALUE; + } + }); + + @TimerId(EVENT_TIMER_ID) + private final TimerSpec eventTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @TimerId(PROCESSING_TIMER_ID) + private final TimerSpec processingTimer = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); + + @ProcessElement + public void dropInput( + ProcessContext context, + BoundedWindow window, + @StateId(BAG_STATE_ID) BagState bagStateState, + @StateId(COMBINING_STATE_ID) CombiningState combiningStateState, + @TimerId(EVENT_TIMER_ID) Timer eventTimerTimer, + @TimerId(PROCESSING_TIMER_ID) Timer processingTimerTimer) { + context.output(null); + } + + @OnTimer(EVENT_TIMER_ID) + public void onEventTime(OnTimerContext context) {} + + @OnTimer(PROCESSING_TIMER_ID) + public void onProcessingTime(OnTimerContext context) {} + + @Override + public boolean equals(Object other) { + return other instanceof StateTimerDropElementsFn; + } + + @Override + public int hashCode() { + return StateTimerDropElementsFn.class.hashCode(); + } + } +} diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto index bf4df2af65fd..c8722e6a39fb 100644 --- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto +++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto @@ -248,10 +248,20 @@ message Parameter { message StateSpec { // TODO: AST for state spec + string id = 1; + Type type = 2; + + enum Type { + VALUE = 0; + BAG = 1; + MAP = 2; + SET = 3; + } } message TimerSpec { // TODO: AST for timer spec + string id = 1; } enum IsBounded { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java index 1103a24a8af1..d48d26b1807c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms.windowing; +import com.google.auto.value.AutoValue; import java.util.Collection; import java.util.Collections; import org.apache.beam.sdk.coders.Coder; @@ -61,10 +62,11 @@ public Coder windowCoder() { @Override public WindowMappingFn getDefaultWindowMappingFn() { - return new GlobalWindowMappingFn(); + return new AutoValue_GlobalWindows_GlobalWindowMappingFn(); } - static class GlobalWindowMappingFn extends WindowMappingFn { + @AutoValue + abstract static class GlobalWindowMappingFn extends WindowMappingFn { @Override public GlobalWindow getSideInputWindow(BoundedWindow mainWindow) { return GlobalWindow.INSTANCE; From 7a2e829c849d3828353994b8eddca4784a900aca Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Mon, 15 May 2017 15:54:03 -0700 Subject: [PATCH 007/159] Splits SplittableParDo into a core-construction part and a KWI-related part --- .../apache/beam/runners/apex/ApexRunner.java | 7 +- .../translation/ApexPipelineTranslator.java | 11 +- .../apex/translation/ParDoTranslator.java | 11 +- .../operators/ApexParDoOperator.java | 10 +- .../construction}/ElementAndRestriction.java | 2 +- .../ElementAndRestrictionCoder.java | 2 +- .../core/construction/SplittableParDo.java | 298 ++++++++++++++ .../ElementAndRestrictionCoderTest.java | 5 +- .../construction/SplittableParDoTest.java | 157 ++++++++ .../apache/beam/runners/core/DoFnRunners.java | 2 +- .../beam/runners/core/ProcessFnRunner.java | 7 +- ... => SplittableParDoViaKeyedWorkItems.java} | 372 +++++------------- ...java => SplittableParDoProcessFnTest.java} | 107 +---- ...tGBKIntoKeyedWorkItemsOverrideFactory.java | 9 +- .../beam/runners/direct/DirectRunner.java | 10 +- .../direct/KeyedPValueTrackingVisitor.java | 4 +- .../direct/ParDoMultiOverrideFactory.java | 2 +- ...ttableProcessElementsEvaluatorFactory.java | 11 +- .../direct/TransformEvaluatorRegistry.java | 4 +- .../FlinkStreamingPipelineTranslator.java | 7 +- .../FlinkStreamingTransformTranslators.java | 29 +- .../streaming/SplittableDoFnOperator.java | 12 +- 22 files changed, 638 insertions(+), 441 deletions(-) rename runners/{core-java/src/main/java/org/apache/beam/runners/core => core-construction-java/src/main/java/org/apache/beam/runners/core/construction}/ElementAndRestriction.java (97%) rename runners/{core-java/src/main/java/org/apache/beam/runners/core => core-construction-java/src/main/java/org/apache/beam/runners/core/construction}/ElementAndRestrictionCoder.java (98%) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java rename runners/{core-java/src/test/java/org/apache/beam/runners/core => core-construction-java/src/test/java/org/apache/beam/runners/core/construction}/ElementAndRestrictionCoderTest.java (96%) create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java rename runners/core-java/src/main/java/org/apache/beam/runners/core/{SplittableParDo.java => SplittableParDoViaKeyedWorkItems.java} (53%) rename runners/core-java/src/test/java/org/apache/beam/runners/core/{SplittableParDoTest.java => SplittableParDoProcessFnTest.java} (82%) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java index 2fd0b22363f5..f91d8e550789 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java @@ -39,12 +39,13 @@ import org.apache.apex.api.Launcher.AppHandle; import org.apache.apex.api.Launcher.LaunchMode; import org.apache.beam.runners.apex.translation.ApexPipelineTranslator; -import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; import org.apache.beam.runners.core.construction.PTransformMatchers; import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.PrimitiveCreate; import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; +import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.coders.Coder; @@ -124,6 +125,10 @@ private List getOverrides() { PTransformOverride.of( PTransformMatchers.splittableParDoMulti(), new SplittableParDoOverrideFactory<>())) + .add( + PTransformOverride.of( + PTransformMatchers.classEqualTo(SplittableParDo.ProcessKeyedElements.class), + new SplittableParDoViaKeyedWorkItems.OverrideFactory<>())) .build(); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java index b3a6d1c0bf64..32e470f6029a 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java @@ -25,7 +25,8 @@ import org.apache.beam.runners.apex.ApexRunner.CreateApexPCollectionView; import org.apache.beam.runners.apex.translation.operators.ApexProcessFnOperator; import org.apache.beam.runners.apex.translation.operators.ApexReadUnboundedInputOperator; -import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.GBKIntoKeyedWorkItems; import org.apache.beam.runners.core.construction.PrimitiveCreate; import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.sdk.Pipeline; @@ -63,9 +64,9 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { static { // register TransformTranslators registerTransformTranslator(ParDo.MultiOutput.class, new ParDoTranslator<>()); - registerTransformTranslator(SplittableParDo.ProcessElements.class, + registerTransformTranslator(SplittableParDoViaKeyedWorkItems.ProcessElements.class, new ParDoTranslator.SplittableProcessElementsTranslator()); - registerTransformTranslator(SplittableParDo.GBKIntoKeyedWorkItems.class, + registerTransformTranslator(GBKIntoKeyedWorkItems.class, new GBKIntoKeyedWorkItemsTranslator()); registerTransformTranslator(Read.Unbounded.class, new ReadUnboundedTranslator()); registerTransformTranslator(Read.Bounded.class, new ReadBoundedTranslator()); @@ -183,11 +184,11 @@ public void translate( } private static class GBKIntoKeyedWorkItemsTranslator - implements TransformTranslator> { + implements TransformTranslator> { @Override public void translate( - SplittableParDo.GBKIntoKeyedWorkItems transform, TranslationContext context) { + GBKIntoKeyedWorkItems transform, TranslationContext context) { // https://issues.apache.org/jira/browse/BEAM-1850 ApexProcessFnOperator> operator = ApexProcessFnOperator.toKeyedWorkItems( context.getPipelineOptions()); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java index 9133cb66f651..e46687a65858 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java @@ -30,7 +30,7 @@ import java.util.Map.Entry; import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.translation.operators.ApexParDoOperator; -import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.ProcessElements; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; @@ -132,14 +132,13 @@ public void translate(ParDo.MultiOutput transform, TranslationC } } - static class SplittableProcessElementsTranslator> - implements TransformTranslator> { + static class SplittableProcessElementsTranslator< + InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker> + implements TransformTranslator> { @Override public void translate( - SplittableParDo.ProcessElements transform, + ProcessElements transform, TranslationContext context) { Map, PValue> outputs = context.getOutputs(); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java index 7fee0d5fcdf4..809ca2a166c5 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexParDoOperator.java @@ -55,7 +55,7 @@ import org.apache.beam.runners.core.SideInputHandler; import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.runners.core.SimplePushbackSideInputDoFnRunner; -import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.ProcessFn; import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StateInternalsFactory; import org.apache.beam.runners.core.StateNamespace; @@ -160,7 +160,7 @@ public ApexParDoOperator( TimerInternals.TimerDataCoder.of(windowingStrategy.getWindowFn().windowCoder()); this.currentKeyTimerInternals = new ApexTimerInternals<>(timerCoder); - if (doFn instanceof SplittableParDo.ProcessFn) { + if (doFn instanceof ProcessFn) { // we know that it is keyed on String Coder keyCoder = StringUtf8Coder.of(); this.currentKeyStateInternals = new StateInternalsProxy<>( @@ -445,15 +445,15 @@ public TimerInternals timerInternals() { pushbackDoFnRunner = SimplePushbackSideInputDoFnRunner.create(doFnRunner, sideInputs, sideInputHandler); - if (doFn instanceof SplittableParDo.ProcessFn) { + if (doFn instanceof ProcessFn) { @SuppressWarnings("unchecked") StateInternalsFactory stateInternalsFactory = (StateInternalsFactory) this.currentKeyStateInternals.getFactory(); @SuppressWarnings({ "rawtypes", "unchecked" }) - SplittableParDo.ProcessFn> - splittableDoFn = (SplittableParDo.ProcessFn) doFn; + ProcessFn> + splittableDoFn = (ProcessFn) doFn; splittableDoFn.setStateInternalsFactory(stateInternalsFactory); TimerInternalsFactory timerInternalsFactory = new TimerInternalsFactory() { @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestriction.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ElementAndRestriction.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestriction.java rename to runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ElementAndRestriction.java index 4a5d0c46168e..53a86b1d4f99 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestriction.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ElementAndRestriction.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core; +package org.apache.beam.runners.core.construction; import com.google.auto.value.AutoValue; import org.apache.beam.sdk.annotations.Experimental; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ElementAndRestrictionCoder.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java rename to runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ElementAndRestrictionCoder.java index 4440b856318b..5ff0aaead9eb 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ElementAndRestrictionCoder.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core; +package org.apache.beam.runners.core.construction; import com.google.common.collect.ImmutableList; import java.io.IOException; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java new file mode 100644 index 000000000000..23ba66ab983d --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import java.util.List; +import java.util.UUID; +import org.apache.beam.sdk.annotations.Experimental; +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.SerializableFunction; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +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.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowingStrategy; + +/** + * A utility transform that executes a splittable {@link DoFn} by expanding it into a + * network of simpler transforms: + * + *

    + *
  1. Pair each element with an initial restriction + *
  2. Split each restriction into sub-restrictions + *
  3. Explode windows, since splitting within each window has to happen independently + *
  4. Assign a unique key to each element/restriction pair + *
  5. Process the keyed element/restriction pairs in a runner-specific way with the splittable + * {@link DoFn}'s {@link DoFn.ProcessElement} method. + *
+ * + *

This transform is intended as a helper for internal use by runners when implementing {@code + * ParDo.of(splittable DoFn)}, but not for direct use by pipeline writers. + */ +@Experimental(Experimental.Kind.SPLITTABLE_DO_FN) +public class SplittableParDo + extends PTransform, PCollectionTuple> { + private final ParDo.MultiOutput parDo; + + /** + * Creates the transform for the given original multi-output {@link ParDo}. + * + * @param parDo The splittable {@link ParDo} transform. + */ + public SplittableParDo(ParDo.MultiOutput parDo) { + checkNotNull(parDo, "parDo must not be null"); + this.parDo = parDo; + checkArgument( + DoFnSignatures.getSignature(parDo.getFn().getClass()).processElement().isSplittable(), + "fn must be a splittable DoFn"); + } + + @Override + public PCollectionTuple expand(PCollection input) { + DoFn fn = parDo.getFn(); + Coder restrictionCoder = + DoFnInvokers.invokerFor(fn) + .invokeGetRestrictionCoder(input.getPipeline().getCoderRegistry()); + Coder> splitCoder = + ElementAndRestrictionCoder.of(input.getCoder(), restrictionCoder); + + PCollection>> keyedRestrictions = + input + .apply( + "Pair with initial restriction", + ParDo.of(new PairWithRestrictionFn(fn))) + .setCoder(splitCoder) + .apply("Split restriction", ParDo.of(new SplitRestrictionFn(fn))) + .setCoder(splitCoder) + // ProcessFn requires all input elements to be in a single window and have a single + // element per work item. This must precede the unique keying so each key has a single + // associated element. + .apply( + "Explode windows", + ParDo.of(new ExplodeWindowsFn>())) + .apply( + "Assign unique key", + WithKeys.of(new RandomUniqueKeyFn>())); + + return keyedRestrictions.apply( + "ProcessKeyedElements", + new ProcessKeyedElements<>( + fn, + input.getCoder(), + restrictionCoder, + (WindowingStrategy) input.getWindowingStrategy(), + parDo.getSideInputs(), + parDo.getMainOutputTag(), + parDo.getAdditionalOutputTags())); + } + + /** + * A {@link DoFn} that forces each of its outputs to be in a single window, by indicating to the + * runner that it observes the window of its input element, so the runner is forced to apply it to + * each input in a single window and thus its output is also in a single window. + */ + private static class ExplodeWindowsFn extends DoFn { + @ProcessElement + public void process(ProcessContext c, BoundedWindow window) { + c.output(c.element()); + } + } + + /** + * Runner-specific primitive {@link PTransform} that invokes the {@link DoFn.ProcessElement} + * method for a splittable {@link DoFn} on each {@link ElementAndRestriction} of the input + * {@link PCollection} of {@link KV KVs} keyed with arbitrary but globally unique keys. + */ + public static class ProcessKeyedElements + extends PTransform< + PCollection>>, PCollectionTuple> { + private final DoFn fn; + private final Coder elementCoder; + private final Coder restrictionCoder; + private final WindowingStrategy windowingStrategy; + private final List> sideInputs; + private final TupleTag mainOutputTag; + private final TupleTagList additionalOutputTags; + + /** + * @param fn the splittable {@link DoFn}. + * @param windowingStrategy the {@link WindowingStrategy} of the input collection. + * @param sideInputs list of side inputs that should be available to the {@link DoFn}. + * @param mainOutputTag {@link TupleTag Tag} of the {@link DoFn DoFn's} main output. + * @param additionalOutputTags {@link TupleTagList Tags} of the {@link DoFn DoFn's} additional + * outputs. + */ + public ProcessKeyedElements( + DoFn fn, + Coder elementCoder, + Coder restrictionCoder, + WindowingStrategy windowingStrategy, + List> sideInputs, + TupleTag mainOutputTag, + TupleTagList additionalOutputTags) { + this.fn = fn; + this.elementCoder = elementCoder; + this.restrictionCoder = restrictionCoder; + this.windowingStrategy = windowingStrategy; + this.sideInputs = sideInputs; + this.mainOutputTag = mainOutputTag; + this.additionalOutputTags = additionalOutputTags; + } + + public DoFn getFn() { + return fn; + } + + public Coder getElementCoder() { + return elementCoder; + } + + public Coder getRestrictionCoder() { + return restrictionCoder; + } + + public WindowingStrategy getInputWindowingStrategy() { + return windowingStrategy; + } + + public List> getSideInputs() { + return sideInputs; + } + + public TupleTag getMainOutputTag() { + return mainOutputTag; + } + + public TupleTagList getAdditionalOutputTags() { + return additionalOutputTags; + } + + @Override + public PCollectionTuple expand( + PCollection>> + input) { + return createPrimitiveOutputFor( + input, fn, mainOutputTag, additionalOutputTags, windowingStrategy); + } + + public static PCollectionTuple createPrimitiveOutputFor( + PCollection input, + DoFn fn, + TupleTag mainOutputTag, + TupleTagList additionalOutputTags, + WindowingStrategy windowingStrategy) { + DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); + PCollectionTuple outputs = + PCollectionTuple.ofPrimitiveOutputsInternal( + input.getPipeline(), + TupleTagList.of(mainOutputTag).and(additionalOutputTags.getAll()), + windowingStrategy, + input.isBounded().and(signature.isBoundedPerElement())); + + // Set output type descriptor similarly to how ParDo.MultiOutput does it. + outputs.get(mainOutputTag).setTypeDescriptor(fn.getOutputTypeDescriptor()); + + return outputs; + } + } + + /** + * Assigns a random unique key to each element of the input collection, so that the output + * collection is effectively the same elements as input, but the per-key state and timers are now + * effectively per-element. + */ + private static class RandomUniqueKeyFn implements SerializableFunction { + @Override + public String apply(T input) { + return UUID.randomUUID().toString(); + } + } + + /** + * Pairs each input element with its initial restriction using the given splittable {@link DoFn}. + */ + private static class PairWithRestrictionFn + extends DoFn> { + private DoFn fn; + private transient DoFnInvoker invoker; + + PairWithRestrictionFn(DoFn fn) { + this.fn = fn; + } + + @Setup + public void setup() { + invoker = DoFnInvokers.invokerFor(fn); + } + + @ProcessElement + public void processElement(ProcessContext context) { + context.output( + ElementAndRestriction.of( + context.element(), + invoker.invokeGetInitialRestriction(context.element()))); + } + } + + /** Splits the restriction using the given {@link SplitRestriction} method. */ + private static class SplitRestrictionFn + extends DoFn< + ElementAndRestriction, + ElementAndRestriction> { + private final DoFn splittableFn; + private transient DoFnInvoker invoker; + + SplitRestrictionFn(DoFn splittableFn) { + this.splittableFn = splittableFn; + } + + @Setup + public void setup() { + invoker = DoFnInvokers.invokerFor(splittableFn); + } + + @ProcessElement + public void processElement(final ProcessContext c) { + final InputT element = c.element().element(); + invoker.invokeSplitRestriction( + element, + c.element().restriction(), + new OutputReceiver() { + @Override + public void output(RestrictionT part) { + c.output(ElementAndRestriction.of(element, part)); + } + }); + } + } +} diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ElementAndRestrictionCoderTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ElementAndRestrictionCoderTest.java similarity index 96% rename from runners/core-java/src/test/java/org/apache/beam/runners/core/ElementAndRestrictionCoderTest.java rename to runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ElementAndRestrictionCoderTest.java index f5160469307d..051cbaa0008a 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ElementAndRestrictionCoderTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ElementAndRestrictionCoderTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.core; +package org.apache.beam.runners.core.construction; import java.util.ArrayList; import java.util.Arrays; @@ -39,8 +39,7 @@ import org.junit.runners.Parameterized.Parameter; /** - * Tests for {@link ElementAndRestrictionCoder}. Parroted from {@link - * org.apache.beam.sdk.coders.KvCoderTest}. + * Tests for {@link ElementAndRestrictionCoder}. */ @RunWith(Parameterized.class) public class ElementAndRestrictionCoderTest { diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java new file mode 100644 index 000000000000..6e4d6c458f04 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import static org.junit.Assert.assertEquals; + +import java.io.Serializable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.BoundedPerElement; +import org.apache.beam.sdk.transforms.DoFn.UnboundedPerElement; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link SplittableParDo}. */ +@RunWith(JUnit4.class) +public class SplittableParDoTest { + // ----------------- Tests for whether the transform sets boundedness correctly -------------- + private static class SomeRestriction + implements Serializable, HasDefaultTracker { + @Override + public SomeRestrictionTracker newTracker() { + return new SomeRestrictionTracker(this); + } + } + + private static class SomeRestrictionTracker implements RestrictionTracker { + private final SomeRestriction someRestriction; + + public SomeRestrictionTracker(SomeRestriction someRestriction) { + this.someRestriction = someRestriction; + } + + @Override + public SomeRestriction currentRestriction() { + return someRestriction; + } + + @Override + public SomeRestriction checkpoint() { + return someRestriction; + } + + @Override + public void checkDone() {} + } + + @BoundedPerElement + private static class BoundedFakeFn extends DoFn { + @ProcessElement + public void processElement(ProcessContext context, SomeRestrictionTracker tracker) {} + + @GetInitialRestriction + public SomeRestriction getInitialRestriction(Integer element) { + return null; + } + } + + @UnboundedPerElement + private static class UnboundedFakeFn extends DoFn { + @ProcessElement + public void processElement(ProcessContext context, SomeRestrictionTracker tracker) {} + + @GetInitialRestriction + public SomeRestriction getInitialRestriction(Integer element) { + return null; + } + } + + private static PCollection makeUnboundedCollection(Pipeline pipeline) { + return pipeline + .apply("unbounded", Create.of(1, 2, 3)) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + } + + private static PCollection makeBoundedCollection(Pipeline pipeline) { + return pipeline + .apply("bounded", Create.of(1, 2, 3)) + .setIsBoundedInternal(PCollection.IsBounded.BOUNDED); + } + + private static final TupleTag MAIN_OUTPUT_TAG = new TupleTag() {}; + + private ParDo.MultiOutput makeParDo(DoFn fn) { + return ParDo.of(fn).withOutputTags(MAIN_OUTPUT_TAG, TupleTagList.empty()); + } + + @Rule + public TestPipeline pipeline = TestPipeline.create(); + + @Test + public void testBoundednessForBoundedFn() { + pipeline.enableAbandonedNodeEnforcement(false); + + DoFn boundedFn = new BoundedFakeFn(); + assertEquals( + "Applying a bounded SDF to a bounded collection produces a bounded collection", + PCollection.IsBounded.BOUNDED, + makeBoundedCollection(pipeline) + .apply("bounded to bounded", new SplittableParDo<>(makeParDo(boundedFn))) + .get(MAIN_OUTPUT_TAG) + .isBounded()); + assertEquals( + "Applying a bounded SDF to an unbounded collection produces an unbounded collection", + PCollection.IsBounded.UNBOUNDED, + makeUnboundedCollection(pipeline) + .apply("bounded to unbounded", new SplittableParDo<>(makeParDo(boundedFn))) + .get(MAIN_OUTPUT_TAG) + .isBounded()); + } + + @Test + public void testBoundednessForUnboundedFn() { + pipeline.enableAbandonedNodeEnforcement(false); + + DoFn unboundedFn = new UnboundedFakeFn(); + assertEquals( + "Applying an unbounded SDF to a bounded collection produces a bounded collection", + PCollection.IsBounded.UNBOUNDED, + makeBoundedCollection(pipeline) + .apply("unbounded to bounded", new SplittableParDo<>(makeParDo(unboundedFn))) + .get(MAIN_OUTPUT_TAG) + .isBounded()); + assertEquals( + "Applying an unbounded SDF to an unbounded collection produces an unbounded collection", + PCollection.IsBounded.UNBOUNDED, + makeUnboundedCollection(pipeline) + .apply("unbounded to unbounded", new SplittableParDo<>(makeParDo(unboundedFn))) + .get(MAIN_OUTPUT_TAG) + .isBounded()); + } +} diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java index f3cca6fcd23b..71dfd11a496d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java @@ -20,7 +20,7 @@ import java.util.Collection; import java.util.List; import org.apache.beam.runners.core.ExecutionContext.StepContext; -import org.apache.beam.runners.core.SplittableParDo.ProcessFn; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.ProcessFn; import org.apache.beam.runners.core.StatefulDoFnRunner.CleanupTimer; import org.apache.beam.runners.core.StatefulDoFnRunner.StateCleaner; import org.apache.beam.sdk.options.PipelineOptions; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java index 61f413fd8cac..31e86bdd0dc5 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java @@ -18,13 +18,13 @@ package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.runners.core.SplittableParDo.ProcessFn; import com.google.common.collect.Iterables; import java.util.Collection; import java.util.Collections; import org.apache.beam.runners.core.StateNamespaces.WindowNamespace; import org.apache.beam.runners.core.TimerInternals.TimerData; +import org.apache.beam.runners.core.construction.ElementAndRestriction; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -32,7 +32,10 @@ import org.apache.beam.sdk.values.PCollectionView; import org.joda.time.Instant; -/** Runs a {@link ProcessFn} by constructing the appropriate contexts and passing them in. */ +/** + * Runs a {@link SplittableParDoViaKeyedWorkItems.ProcessFn} by constructing the appropriate + * contexts and passing them in. + */ public class ProcessFnRunner implements PushbackSideInputDoFnRunner< KeyedWorkItem>, OutputT> { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java similarity index 53% rename from runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java index 6503fa20938d..b38e364c9856 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java @@ -17,159 +17,48 @@ */ package org.apache.beam.runners.core; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; - import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterables; import java.util.List; -import java.util.UUID; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.CannotProvideCoderException; +import java.util.Map; +import org.apache.beam.runners.core.construction.ElementAndRestriction; +import org.apache.beam.runners.core.construction.PTransformReplacements; +import org.apache.beam.runners.core.construction.ReplacementOutputs; +import org.apache.beam.runners.core.construction.SplittableParDo.ProcessKeyedElements; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.DoFn; 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.WithKeys; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.transforms.reflect.DoFnSignature; -import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; 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.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** - * A utility transform that executes a splittable {@link DoFn} by expanding it into a - * network of simpler transforms: - * - *

    - *
  1. Pair each element with an initial restriction - *
  2. Split each restriction into sub-restrictions - *
  3. Assign a unique key to each element/restriction pair - *
  4. Group by key (so that work is partitioned by key and we can access state/timers) - *
  5. Process each keyed element/restriction pair with the splittable {@link DoFn}'s {@link - * DoFn.ProcessElement} method, using state and timers API. - *
- * - *

This transform is intended as a helper for internal use by runners when implementing {@code - * ParDo.of(splittable DoFn)}, but not for direct use by pipeline writers. + * Utilities for implementing {@link ProcessKeyedElements} using {@link KeyedWorkItem} and + * runner-specific {@link StateInternals} and {@link TimerInternals}. */ -@Experimental(Experimental.Kind.SPLITTABLE_DO_FN) -public class SplittableParDo - extends PTransform, PCollectionTuple> { - private final ParDo.MultiOutput parDo; - - /** - * Creates the transform for the given original multi-output {@link ParDo}. - * - * @param parDo The splittable {@link ParDo} transform. - */ - public SplittableParDo(ParDo.MultiOutput parDo) { - checkNotNull(parDo, "parDo must not be null"); - this.parDo = parDo; - checkArgument( - DoFnSignatures.getSignature(parDo.getFn().getClass()).processElement().isSplittable(), - "fn must be a splittable DoFn"); - } - - @Override - public PCollectionTuple expand(PCollection input) { - return applyTyped(input); - } - - private PCollectionTuple applyTyped(PCollection input) { - DoFn fn = parDo.getFn(); - Coder restrictionCoder = - DoFnInvokers.invokerFor(fn) - .invokeGetRestrictionCoder(input.getPipeline().getCoderRegistry()); - PCollection>> keyedWorkItems = - applySplitIntoKeyedWorkItems(input, fn, restrictionCoder); - return keyedWorkItems.apply( - "Process", - new ProcessElements<>( - fn, - input.getCoder(), - restrictionCoder, - (WindowingStrategy) input.getWindowingStrategy(), - parDo.getSideInputs(), - parDo.getMainOutputTag(), - parDo.getAdditionalOutputTags())); - } - - private static - PCollection>> - applySplitIntoKeyedWorkItems( - PCollection input, - DoFn fn, - Coder restrictionCoder) { - Coder> splitCoder = - ElementAndRestrictionCoder.of(input.getCoder(), restrictionCoder); - - PCollection>> keyedWorkItems = - input - .apply( - "Pair with initial restriction", - ParDo.of(new PairWithRestrictionFn(fn))) - .setCoder(splitCoder) - .apply("Split restriction", ParDo.of(new SplitRestrictionFn(fn))) - .setCoder(splitCoder) - // ProcessFn requires all input elements to be in a single window and have a single - // element per work item. This must precede the unique keying so each key has a single - // associated element. - .apply( - "Explode windows", - ParDo.of(new ExplodeWindowsFn>())) - .apply( - "Assign unique key", - WithKeys.of(new RandomUniqueKeyFn>())) - .apply( - "Group by key", - new GBKIntoKeyedWorkItems>()) - .setCoder( - KeyedWorkItemCoder.of( - StringUtf8Coder.of(), - splitCoder, - input.getWindowingStrategy().getWindowFn().windowCoder())); - checkArgument( - keyedWorkItems.getWindowingStrategy().getWindowFn() instanceof GlobalWindows, - "GBKIntoKeyedWorkItems must produce a globally windowed collection, " - + "but windowing strategy was: %s", - keyedWorkItems.getWindowingStrategy()); - return keyedWorkItems; - } - - /** - * A {@link DoFn} that forces each of its outputs to be in a single window, by indicating to the - * runner that it observes the window of its input element, so the runner is forced to apply it to - * each input in a single window and thus its output is also in a single window. - */ - private static class ExplodeWindowsFn extends DoFn { - @ProcessElement - public void process(ProcessContext c, BoundedWindow window) { - c.output(c.element()); - } - } - +public class SplittableParDoViaKeyedWorkItems { /** * Runner-specific primitive {@link GroupByKey GroupByKey-like} {@link PTransform} that produces * {@link KeyedWorkItem KeyedWorkItems} so that downstream transforms can access state and timers. @@ -186,143 +75,106 @@ public PCollection> expand(PCollection + implements PTransformOverrideFactory< + PCollection>>, PCollectionTuple, + ProcessKeyedElements> { + @Override + public PTransformReplacement< + PCollection>>, PCollectionTuple> + getReplacementTransform( + AppliedPTransform< + PCollection>>, + PCollectionTuple, ProcessKeyedElements> + transform) { + return PTransformReplacement.of( + PTransformReplacements.getSingletonMainInput(transform), + new SplittableProcessViaKeyedWorkItems<>(transform.getTransform())); + } + + @Override + public Map mapOutputs( + Map, PValue> outputs, PCollectionTuple newOutput) { + return ReplacementOutputs.tagged(outputs, newOutput); + } + } + /** * Runner-specific primitive {@link PTransform} that invokes the {@link DoFn.ProcessElement} * method for a splittable {@link DoFn}. */ - public static class ProcessElements< - InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker> + public static class SplittableProcessViaKeyedWorkItems extends PTransform< - PCollection>>, - PCollectionTuple> { - private final DoFn fn; - private final Coder elementCoder; - private final Coder restrictionCoder; - private final WindowingStrategy windowingStrategy; - private final List> sideInputs; - private final TupleTag mainOutputTag; - private final TupleTagList additionalOutputTags; - - /** - * @param fn the splittable {@link DoFn}. - * @param windowingStrategy the {@link WindowingStrategy} of the input collection. - * @param sideInputs list of side inputs that should be available to the {@link DoFn}. - * @param mainOutputTag {@link TupleTag Tag} of the {@link DoFn DoFn's} main output. - * @param additionalOutputTags {@link TupleTagList Tags} of the {@link DoFn DoFn's} additional - * outputs. - */ - public ProcessElements( - DoFn fn, - Coder elementCoder, - Coder restrictionCoder, - WindowingStrategy windowingStrategy, - List> sideInputs, - TupleTag mainOutputTag, - TupleTagList additionalOutputTags) { - this.fn = fn; - this.elementCoder = elementCoder; - this.restrictionCoder = restrictionCoder; - this.windowingStrategy = windowingStrategy; - this.sideInputs = sideInputs; - this.mainOutputTag = mainOutputTag; - this.additionalOutputTags = additionalOutputTags; - } + PCollection>>, PCollectionTuple> { + private final ProcessKeyedElements original; - public DoFn getFn() { - return fn; + public SplittableProcessViaKeyedWorkItems( + ProcessKeyedElements original) { + this.original = original; } - public List> getSideInputs() { - return sideInputs; + @Override + public PCollectionTuple expand( + PCollection>> input) { + return input + .apply(new GBKIntoKeyedWorkItems>()) + .setCoder( + KeyedWorkItemCoder.of( + StringUtf8Coder.of(), + ((KvCoder>) input.getCoder()) + .getValueCoder(), + input.getWindowingStrategy().getWindowFn().windowCoder())) + .apply(new ProcessElements<>(original)); } + } - public TupleTag getMainOutputTag() { - return mainOutputTag; - } + /** A primitive transform wrapping around {@link ProcessFn}. */ + public static class ProcessElements< + InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker> + extends PTransform< + PCollection>>, + PCollectionTuple> { + private final ProcessKeyedElements original; - public TupleTagList getAdditionalOutputTags() { - return additionalOutputTags; + public ProcessElements(ProcessKeyedElements original) { + this.original = original; } public ProcessFn newProcessFn( DoFn fn) { - return new SplittableParDo.ProcessFn<>( - fn, elementCoder, restrictionCoder, windowingStrategy); - } - - @Override - public PCollectionTuple expand( - PCollection>> - input) { - DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); - PCollectionTuple outputs = - PCollectionTuple.ofPrimitiveOutputsInternal( - input.getPipeline(), - TupleTagList.of(mainOutputTag).and(additionalOutputTags.getAll()), - windowingStrategy, - input.isBounded().and(signature.isBoundedPerElement())); - - // Set output type descriptor similarly to how ParDo.MultiOutput does it. - outputs.get(mainOutputTag).setTypeDescriptor(fn.getOutputTypeDescriptor()); - - return outputs; + return new ProcessFn<>( + fn, + original.getElementCoder(), + original.getRestrictionCoder(), + original.getInputWindowingStrategy()); } - @Override - public Coder getDefaultOutputCoder( - PCollection>> - input, - PCollection output) - throws CannotProvideCoderException { - // Similar logic to ParDo.MultiOutput.getOutputCoder. - @SuppressWarnings("unchecked") - KeyedWorkItemCoder> kwiCoder = - (KeyedWorkItemCoder) input.getCoder(); - Coder inputCoder = - ((ElementAndRestrictionCoder) kwiCoder.getElementCoder()) - .getElementCoder(); - return input - .getPipeline() - .getCoderRegistry() - .getCoder(output.getTypeDescriptor(), fn.getInputTypeDescriptor(), inputCoder); + public DoFn getFn() { + return original.getFn(); } - } - /** - * Assigns a random unique key to each element of the input collection, so that the output - * collection is effectively the same elements as input, but the per-key state and timers are now - * effectively per-element. - */ - private static class RandomUniqueKeyFn implements SerializableFunction { - @Override - public String apply(T input) { - return UUID.randomUUID().toString(); + public List> getSideInputs() { + return original.getSideInputs(); } - } - /** - * Pairs each input element with its initial restriction using the given splittable {@link DoFn}. - */ - private static class PairWithRestrictionFn - extends DoFn> { - private DoFn fn; - private transient DoFnInvoker invoker; - - PairWithRestrictionFn(DoFn fn) { - this.fn = fn; + public TupleTag getMainOutputTag() { + return original.getMainOutputTag(); } - @Setup - public void setup() { - invoker = DoFnInvokers.invokerFor(fn); + public TupleTagList getAdditionalOutputTags() { + return original.getAdditionalOutputTags(); } - @ProcessElement - public void processElement(ProcessContext context) { - context.output( - ElementAndRestriction.of( - context.element(), - invoker.invokeGetInitialRestriction(context.element()))); + @Override + public PCollectionTuple expand( + PCollection>> input) { + return ProcessKeyedElements.createPrimitiveOutputFor( + input, + original.getFn(), + original.getMainOutputTag(), + original.getAdditionalOutputTags(), + original.getInputWindowingStrategy()); } } @@ -331,10 +183,10 @@ public void processElement(ProcessContext context) { * by creating a tracker for the restriction and checkpointing/resuming processing later if * necessary. * - *

Takes {@link KeyedWorkItem} and assumes that the KeyedWorkItem contains a single element - * (or a single timer set by {@link ProcessFn itself}, in a single window. This is necessary - * because {@link ProcessFn} sets timers, and timers are namespaced to a single window and it - * should be the window of the input element. + *

Takes {@link KeyedWorkItem} and assumes that the KeyedWorkItem contains a single element (or + * a single timer set by {@link ProcessFn itself}, in a single window. This is necessary because + * {@link ProcessFn} sets timers, and timers are namespaced to a single window and it should be + * the window of the input element. * *

See also: https://issues.apache.org/jira/browse/BEAM-1983 */ @@ -354,8 +206,7 @@ public static class ProcessFn< */ private static final StateTag watermarkHoldTag = StateTags.makeSystemTagInternal( - StateTags.watermarkStateInternal( - "hold", TimestampCombiner.LATEST)); + StateTags.watermarkStateInternal("hold", TimestampCombiner.LATEST)); /** * The state cell containing a copy of the element. Written during the first {@link @@ -452,8 +303,7 @@ public void finishBundle(FinishBundleContext c) throws Exception { @ProcessElement public void processElement(final ProcessContext c) { String key = c.element().key(); - StateInternals stateInternals = - stateInternalsFactory.stateInternalsForKey(key); + StateInternals stateInternals = stateInternalsFactory.stateInternalsForKey(key); TimerInternals timerInternals = timerInternalsFactory.timerInternalsForKey(key); // Initialize state (element and restriction) depending on whether this is the seed call. @@ -477,8 +327,7 @@ public void processElement(final ProcessContext c) { stateInternals.state(stateNamespace, elementTag); ValueState restrictionState = stateInternals.state(stateNamespace, restrictionTag); - WatermarkHoldState holdState = - stateInternals.state(stateNamespace, watermarkHoldTag); + WatermarkHoldState holdState = stateInternals.state(stateNamespace, watermarkHoldTag); ElementAndRestriction, RestrictionT> elementAndRestriction; if (isSeedCall) { @@ -565,38 +414,5 @@ private void throwUnsupportedOutput() { } }; } - - } - - /** Splits the restriction using the given {@link DoFn.SplitRestriction} method. */ - private static class SplitRestrictionFn - extends DoFn< - ElementAndRestriction, - ElementAndRestriction> { - private final DoFn splittableFn; - private transient DoFnInvoker invoker; - - SplitRestrictionFn(DoFn splittableFn) { - this.splittableFn = splittableFn; - } - - @Setup - public void setup() { - invoker = DoFnInvokers.invokerFor(splittableFn); - } - - @ProcessElement - public void processElement(final ProcessContext c) { - final InputT element = c.element().element(); - invoker.invokeSplitRestriction( - element, - c.element().restriction(), - new OutputReceiver() { - @Override - public void output(RestrictionT part) { - c.output(ElementAndRestriction.of(element, part)); - } - }); - } } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java similarity index 82% rename from runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java index be4cf08b662d..d2424318eea5 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java @@ -34,19 +34,15 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.Executors; -import javax.annotation.Nullable; -import org.apache.beam.sdk.Pipeline; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.ProcessFn; +import org.apache.beam.runners.core.construction.ElementAndRestriction; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.BoundedPerElement; -import org.apache.beam.sdk.transforms.DoFn.UnboundedPerElement; import org.apache.beam.sdk.transforms.DoFnTester; -import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRange; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; @@ -57,11 +53,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.ValueInSingleWindow; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; @@ -71,9 +65,9 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** Tests for {@link SplittableParDo}. */ +/** Tests for {@link SplittableParDoViaKeyedWorkItems.ProcessFn}. */ @RunWith(JUnit4.class) -public class SplittableParDoTest { +public class SplittableParDoProcessFnTest { private static final int MAX_OUTPUTS_PER_BUNDLE = 10000; private static final Duration MAX_BUNDLE_DURATION = Duration.standardSeconds(5); @@ -107,96 +101,12 @@ public SomeRestriction checkpoint() { public void checkDone() {} } - @BoundedPerElement - private static class BoundedFakeFn extends DoFn { - @ProcessElement - public void processElement(ProcessContext context, SomeRestrictionTracker tracker) {} - - @GetInitialRestriction - public SomeRestriction getInitialRestriction(Integer element) { - return null; - } - } - - @UnboundedPerElement - private static class UnboundedFakeFn extends DoFn { - @ProcessElement - public void processElement(ProcessContext context, SomeRestrictionTracker tracker) {} - - @GetInitialRestriction - public SomeRestriction getInitialRestriction(Integer element) { - return null; - } - } - - private static PCollection makeUnboundedCollection(Pipeline pipeline) { - return pipeline - .apply("unbounded", Create.of(1, 2, 3)) - .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); - } - - private static PCollection makeBoundedCollection(Pipeline pipeline) { - return pipeline - .apply("bounded", Create.of(1, 2, 3)) - .setIsBoundedInternal(PCollection.IsBounded.BOUNDED); - } - - private static final TupleTag MAIN_OUTPUT_TAG = new TupleTag() {}; - - private ParDo.MultiOutput makeParDo(DoFn fn) { - return ParDo.of(fn).withOutputTags(MAIN_OUTPUT_TAG, TupleTagList.empty()); - } - @Rule public TestPipeline pipeline = TestPipeline.create(); - @Test - public void testBoundednessForBoundedFn() { - pipeline.enableAbandonedNodeEnforcement(false); - - DoFn boundedFn = new BoundedFakeFn(); - assertEquals( - "Applying a bounded SDF to a bounded collection produces a bounded collection", - PCollection.IsBounded.BOUNDED, - makeBoundedCollection(pipeline) - .apply("bounded to bounded", new SplittableParDo<>(makeParDo(boundedFn))) - .get(MAIN_OUTPUT_TAG) - .isBounded()); - assertEquals( - "Applying a bounded SDF to an unbounded collection produces an unbounded collection", - PCollection.IsBounded.UNBOUNDED, - makeUnboundedCollection(pipeline) - .apply("bounded to unbounded", new SplittableParDo<>(makeParDo(boundedFn))) - .get(MAIN_OUTPUT_TAG) - .isBounded()); - } - - @Test - public void testBoundednessForUnboundedFn() { - pipeline.enableAbandonedNodeEnforcement(false); - - DoFn unboundedFn = new UnboundedFakeFn(); - assertEquals( - "Applying an unbounded SDF to a bounded collection produces a bounded collection", - PCollection.IsBounded.UNBOUNDED, - makeBoundedCollection(pipeline) - .apply("unbounded to bounded", new SplittableParDo<>(makeParDo(unboundedFn))) - .get(MAIN_OUTPUT_TAG) - .isBounded()); - assertEquals( - "Applying an unbounded SDF to an unbounded collection produces an unbounded collection", - PCollection.IsBounded.UNBOUNDED, - makeUnboundedCollection(pipeline) - .apply("unbounded to unbounded", new SplittableParDo<>(makeParDo(unboundedFn))) - .get(MAIN_OUTPUT_TAG) - .isBounded()); - } - - // ------------------------------- Tests for ProcessFn --------------------------------- - /** - * A helper for testing {@link SplittableParDo.ProcessFn} on 1 element (but possibly over multiple - * {@link DoFn.ProcessElement} calls). + * A helper for testing {@link ProcessFn} on 1 element (but + * possibly over multiple {@link DoFn.ProcessElement} calls). */ private static class ProcessFnTester< InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker> @@ -221,8 +131,8 @@ private static class ProcessFnTester< // encode IntervalWindow's because that's what all tests here use. WindowingStrategy windowingStrategy = (WindowingStrategy) WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(1))); - final SplittableParDo.ProcessFn processFn = - new SplittableParDo.ProcessFn<>( + final ProcessFn processFn = + new ProcessFn<>( fn, inputCoder, restrictionCoder, windowingStrategy); this.tester = DoFnTester.of(processFn); this.timerInternals = new InMemoryTimerInternals(); @@ -248,7 +158,6 @@ public TimerInternals timerInternalsForKey(String key) { tester.getPipelineOptions(), new OutputWindowedValueToDoFnTester<>(tester), new SideInputReader() { - @Nullable @Override public T get(PCollectionView view, BoundedWindow window) { throw new NoSuchElementException(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java index 64eecc852afd..3fefe206b2df 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java @@ -18,17 +18,14 @@ package org.apache.beam.runners.direct; import org.apache.beam.runners.core.KeyedWorkItem; -import org.apache.beam.runners.core.SplittableParDo.GBKIntoKeyedWorkItems; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.GBKIntoKeyedWorkItems; import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -/** - * Provides an implementation of {@link SplittableParDo.GBKIntoKeyedWorkItems} for the Direct - * Runner. - */ +/** Provides an implementation of {@link GBKIntoKeyedWorkItems} for the Direct Runner. */ class DirectGBKIntoKeyedWorkItemsOverrideFactory extends SingleInputOutputOverrideFactory< PCollection>, PCollection>, @@ -39,7 +36,7 @@ class DirectGBKIntoKeyedWorkItemsOverrideFactory getReplacementTransform( AppliedPTransform< PCollection>, PCollection>, - GBKIntoKeyedWorkItems> + GBKIntoKeyedWorkItems> transform) { return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index 181896f3b3fc..69dea8fa4cb6 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -27,8 +27,9 @@ import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.core.SplittableParDo.GBKIntoKeyedWorkItems; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; import org.apache.beam.runners.core.construction.PTransformMatchers; +import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory; import org.apache.beam.sdk.Pipeline; @@ -244,7 +245,12 @@ private List defaultTransformOverrides() { PTransformMatchers.stateOrTimerParDoMulti(), new ParDoMultiOverrideFactory())) .add( PTransformOverride.of( - PTransformMatchers.classEqualTo(GBKIntoKeyedWorkItems.class), + PTransformMatchers.classEqualTo(SplittableParDo.ProcessKeyedElements.class), + new SplittableParDoViaKeyedWorkItems.OverrideFactory())) + .add( + PTransformOverride.of( + PTransformMatchers.classEqualTo( + SplittableParDoViaKeyedWorkItems.GBKIntoKeyedWorkItems.class), new DirectGBKIntoKeyedWorkItemsOverrideFactory())) /* Returns a GBKO */ .add( PTransformOverride.of( diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java index f9b6eba215f2..347f313af85a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java @@ -23,7 +23,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.sdk.Pipeline.PipelineVisitor; @@ -48,7 +48,7 @@ class KeyedPValueTrackingVisitor implements PipelineVisitor { private static final Set> PRODUCES_KEYED_OUTPUTS = ImmutableSet.of( - SplittableParDo.GBKIntoKeyedWorkItems.class, + SplittableParDoViaKeyedWorkItems.GBKIntoKeyedWorkItems.class, DirectGroupByKeyOnly.class, DirectGroupAlsoByWindow.class); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index 89903da1ce00..be433db1ca00 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -23,9 +23,9 @@ import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItemCoder; import org.apache.beam.runners.core.KeyedWorkItems; -import org.apache.beam.runners.core.SplittableParDo; import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.ReplacementOutputs; +import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index f490b0ba49d0..dc85d87bc93f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -24,18 +24,18 @@ import java.util.concurrent.Executors; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.DoFnRunners.OutputManager; -import org.apache.beam.runners.core.ElementAndRestriction; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.OutputAndTimeBoundedSplittableProcessElementInvoker; import org.apache.beam.runners.core.OutputWindowedValue; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.runners.core.ReadyCheckingSideInputReader; -import org.apache.beam.runners.core.SplittableParDo; -import org.apache.beam.runners.core.SplittableParDo.ProcessFn; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.ProcessElements; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.ProcessFn; import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StateInternalsFactory; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternalsFactory; +import org.apache.beam.runners.core.construction.ElementAndRestriction; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; @@ -88,12 +88,11 @@ public void cleanup() throws Exception { createEvaluator( AppliedPTransform< PCollection>>, - PCollectionTuple, - SplittableParDo.ProcessElements> + PCollectionTuple, ProcessElements> application, CommittedBundle inputBundle) throws Exception { - final SplittableParDo.ProcessElements transform = + final ProcessElements transform = application.getTransform(); ProcessFn processFn = diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java index d0e622d7f934..718cca2f5ddb 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java @@ -25,7 +25,7 @@ import java.util.Collection; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; @@ -66,7 +66,7 @@ public static TransformEvaluatorRegistry defaultRegistry(EvaluationContext ctxt) new TestStreamEvaluatorFactory(ctxt)) // Runner-specific primitive used in expansion of SplittableParDo .put( - SplittableParDo.ProcessElements.class, + SplittableParDoViaKeyedWorkItems.ProcessElements.class, new SplittableProcessElementsEvaluatorFactory<>(ctxt)) .build(); return new TransformEvaluatorRegistry(primitives); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java index 35d1bcd19f68..53a1fa160ad1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java @@ -20,11 +20,12 @@ import com.google.common.collect.ImmutableList; import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; import org.apache.beam.runners.core.construction.PTransformMatchers; import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; +import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.runners.core.construction.UnconsumedReads; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; @@ -78,6 +79,10 @@ public void translate(Pipeline pipeline) { PTransformOverride.of( PTransformMatchers.splittableParDoMulti(), new SplittableParDoOverrideFactory())) + .add( + PTransformOverride.of( + PTransformMatchers.classEqualTo(SplittableParDo.ProcessKeyedElements.class), + new SplittableParDoViaKeyedWorkItems.OverrideFactory())) .add( PTransformOverride.of( PTransformMatchers.classEqualTo(View.AsIterable.class), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java index 9a9320564739..00e9934aacbf 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java @@ -30,10 +30,10 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import org.apache.beam.runners.core.ElementAndRestriction; import org.apache.beam.runners.core.KeyedWorkItem; -import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; import org.apache.beam.runners.core.SystemReduceFn; +import org.apache.beam.runners.core.construction.ElementAndRestriction; import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator; @@ -119,9 +119,11 @@ class FlinkStreamingTransformTranslators { TRANSLATORS.put(ParDo.MultiOutput.class, new ParDoStreamingTranslator()); TRANSLATORS.put( - SplittableParDo.ProcessElements.class, new SplittableProcessElementsStreamingTranslator()); + SplittableParDoViaKeyedWorkItems.ProcessElements.class, + new SplittableProcessElementsStreamingTranslator()); TRANSLATORS.put( - SplittableParDo.GBKIntoKeyedWorkItems.class, new GBKIntoKeyedWorkItemsTranslator()); + SplittableParDoViaKeyedWorkItems.GBKIntoKeyedWorkItems.class, + new GBKIntoKeyedWorkItemsTranslator()); TRANSLATORS.put(Window.Assign.class, new WindowAssignTranslator()); @@ -329,7 +331,8 @@ public RawUnionValue map(T o) throws Exception { } /** - * Helper for translating {@link ParDo.MultiOutput} and {@link SplittableParDo.ProcessElements}. + * Helper for translating {@link ParDo.MultiOutput} and {@link + * SplittableParDoViaKeyedWorkItems.ProcessElements}. */ static class ParDoTranslationHelper { @@ -382,7 +385,7 @@ static void translateParDo( keyCoder = ((KvCoder) input.getCoder()).getKeyCoder(); inputDataStream = inputDataStream.keyBy(new KvToByteBufferKeySelector(keyCoder)); stateful = true; - } else if (doFn instanceof SplittableParDo.ProcessFn) { + } else if (doFn instanceof SplittableParDoViaKeyedWorkItems.ProcessFn) { // we know that it is keyed on String keyCoder = StringUtf8Coder.of(); stateful = true; @@ -578,19 +581,19 @@ public DoFnOperator createDoFnOperator( private static class SplittableProcessElementsStreamingTranslator< InputT, OutputT, RestrictionT, TrackerT extends RestrictionTracker> extends FlinkStreamingPipelineTranslator.StreamTransformTranslator< - SplittableParDo.ProcessElements> { + SplittableParDoViaKeyedWorkItems.ProcessElements> { @Override public void translateNode( - SplittableParDo.ProcessElements transform, + SplittableParDoViaKeyedWorkItems.ProcessElements + transform, FlinkStreamingTranslationContext context) { ParDoTranslationHelper.translateParDo( transform.getName(), transform.newProcessFn(transform.getFn()), context.getCurrentTransform().getFullName(), - (PCollection>>) - context.getInput(transform), + context.getInput(transform), transform.getSideInputs(), context.getOutputs(transform), transform.getMainOutputTag(), @@ -930,18 +933,18 @@ public void translateNode( private static class GBKIntoKeyedWorkItemsTranslator extends FlinkStreamingPipelineTranslator.StreamTransformTranslator< - SplittableParDo.GBKIntoKeyedWorkItems> { + SplittableParDoViaKeyedWorkItems.GBKIntoKeyedWorkItems> { @Override boolean canTranslate( - SplittableParDo.GBKIntoKeyedWorkItems transform, + SplittableParDoViaKeyedWorkItems.GBKIntoKeyedWorkItems transform, FlinkStreamingTranslationContext context) { return true; } @Override public void translateNode( - SplittableParDo.GBKIntoKeyedWorkItems transform, + SplittableParDoViaKeyedWorkItems.GBKIntoKeyedWorkItems transform, FlinkStreamingTranslationContext context) { PCollection> input = context.getInput(transform); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java index 968fc0a540a7..4ac2ff522138 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java @@ -26,16 +26,16 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import org.apache.beam.runners.core.ElementAndRestriction; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.core.OutputAndTimeBoundedSplittableProcessElementInvoker; import org.apache.beam.runners.core.OutputWindowedValue; -import org.apache.beam.runners.core.SplittableParDo; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.ProcessFn; import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StateInternalsFactory; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternalsFactory; +import org.apache.beam.runners.core.construction.ElementAndRestriction; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; @@ -94,7 +94,7 @@ public SplittableDoFnOperator( public void open() throws Exception { super.open(); - checkState(doFn instanceof SplittableParDo.ProcessFn); + checkState(doFn instanceof ProcessFn); StateInternalsFactory stateInternalsFactory = new StateInternalsFactory() { @Override @@ -114,9 +114,9 @@ public TimerInternals timerInternalsForKey(String key) { executorService = Executors.newSingleThreadScheduledExecutor(Executors.defaultThreadFactory()); - ((SplittableParDo.ProcessFn) doFn).setStateInternalsFactory(stateInternalsFactory); - ((SplittableParDo.ProcessFn) doFn).setTimerInternalsFactory(timerInternalsFactory); - ((SplittableParDo.ProcessFn) doFn).setProcessElementInvoker( + ((ProcessFn) doFn).setStateInternalsFactory(stateInternalsFactory); + ((ProcessFn) doFn).setTimerInternalsFactory(timerInternalsFactory); + ((ProcessFn) doFn).setProcessElementInvoker( new OutputAndTimeBoundedSplittableProcessElementInvoker<>( doFn, serializedOptions.getPipelineOptions(), From 54808be14ff2e86088807adc3c0dc0965c8a6f03 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 18 May 2017 18:43:31 -0400 Subject: [PATCH 008/159] [BEAM-2318] Ensure that tables that don't exist are not created We use the same test table name in two tests: one that expects the table not to exist, another that creates the table. Obviously, the correctness of this will depend on the order in which unit tests are executed. Fix the flake by using different and more explicit test names. --- .../test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java index dbeab04c9731..1cdfc7f0096b 100644 --- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java +++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java @@ -282,7 +282,7 @@ public void testWriting() throws Exception { /** Tests that when writing to a non-existent table, the write fails. */ @Test public void testWritingFailsTableDoesNotExist() throws Exception { - final String table = "TEST-TABLE"; + final String table = "TEST-TABLE-DOES-NOT-EXIST"; PCollection>> emptyInput = p.apply(Create.empty(HBaseIO.WRITE_CODER)); @@ -298,7 +298,7 @@ public void testWritingFailsTableDoesNotExist() throws Exception { /** Tests that when writing an element fails, the write fails. */ @Test public void testWritingFailsBadElement() throws Exception { - final String table = "TEST-TABLE"; + final String table = "TEST-TABLE-BAD-ELEMENT"; final String key = "KEY"; createTable(table); From 1da8da79dcc623c53b35d97419566b736447f6b6 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 18 May 2017 16:19:29 -0400 Subject: [PATCH 009/159] [BEAM-2391] Clone Scan in HBaseReader --- .../org/apache/beam/sdk/io/hbase/HBaseIO.java | 5 +++-- .../apache/beam/sdk/io/hbase/HBaseIOTest.java | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java index 3c42da91f27a..849873c05970 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java @@ -487,8 +487,9 @@ public boolean start() throws IOException { connection = ConnectionFactory.createConnection(configuration); TableName tableName = TableName.valueOf(tableId); Table table = connection.getTable(tableName); - Scan scan = source.read.serializableScan.get(); - scanner = table.getScanner(scan); + // [BEAM-2319] We have to clone the Scan because the underlying scanner may mutate it. + Scan scanClone = new Scan(source.read.serializableScan.get()); + scanner = table.getScanner(scanClone); iter = scanner.iterator(); return advance(); } diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java index 1cdfc7f0096b..4a067895b22c 100644 --- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java +++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java @@ -33,6 +33,7 @@ import org.apache.beam.sdk.io.range.ByteKey; import org.apache.beam.sdk.io.range.ByteKeyRange; import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; @@ -205,6 +206,22 @@ public void testReadingWithSplits() throws Exception { assertSourcesEqualReferenceSource(source, splits, null /* options */); } + /** Tests that a {@link HBaseSource} can be read twice, verifying its immutability. */ + @Test + public void testReadingSourceTwice() throws Exception { + final String table = "TEST-READING-TWICE"; + final int numRows = 10; + + // Set up test table data and sample row keys for size estimation and splitting. + createTable(table); + writeData(table, numRows); + + HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); + HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */); + assertThat(SourceTestUtils.readFromSource(source, null), hasSize(numRows)); + // second read. + assertThat(SourceTestUtils.readFromSource(source, null), hasSize(numRows)); + } /** Tests reading all rows using a filter. */ @Test From f969ea443c90345cf8c33f953c399e46223e8d03 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Mon, 15 May 2017 13:39:12 -0700 Subject: [PATCH 010/159] [BEAM-2252] Use pluginManagement instead of redefining maven-shade-plugin to inherit build order from root pom.xml --- runners/direct-java/pom.xml | 175 ++++++++++----------- runners/google-cloud-dataflow-java/pom.xml | 121 +++++++------- sdks/java/core/pom.xml | 165 ++++++++++--------- sdks/java/io/google-cloud-platform/pom.xml | 33 ++-- 4 files changed, 237 insertions(+), 257 deletions(-) diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index c581113120dd..857dcf42fd7b 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -40,6 +40,92 @@ + + + + org.apache.maven.plugins + maven-shade-plugin + + + bundle-and-repackage + package + + shade + + + true + + + com.google.guava:guava + com.google.protobuf:protobuf-java + org.apache.beam:beam-runners-core-construction-java + org.apache.beam:beam-runners-core-java + org.apache.beam:beam-sdks-common-runner-api + com.google.code.findbugs:jsr305 + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + org.apache.beam.runners.core + + org.apache.beam.runners.direct.repackaged.runners.core + + + + org.apache.beam.sdk.common + + org.apache.beam.runners.direct.repackaged.sdk.common + + + + com.google.common + + + com.google.common.**.testing.* + + + org.apache.beam.runners.direct.repackaged.com.google.common + + + + com.google.protobuf + + org.apache.beam.runners.direct.repackaged.com.google.protobuf + + + + com.google.thirdparty + + org.apache.beam.runners.direct.repackaged.com.google.thirdparty + + + + javax.annotation + + org.apache.beam.runners.direct.repackaged.javax.annotation + + + + + + + + + + + + + org.apache.maven.plugins @@ -77,95 +163,6 @@ - - - org.apache.maven.plugins - maven-jar-plugin - - - - org.apache.maven.plugins - maven-shade-plugin - - - bundle-and-repackage - package - - shade - - - true - - - com.google.guava:guava - com.google.protobuf:protobuf-java - org.apache.beam:beam-runners-core-construction-java - org.apache.beam:beam-runners-core-java - org.apache.beam:beam-sdks-common-runner-api - com.google.code.findbugs:jsr305 - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - org.apache.beam.runners.core - - org.apache.beam.runners.direct.repackaged.runners.core - - - - org.apache.beam.sdk.common - - org.apache.beam.runners.direct.repackaged.sdk.common - - - - com.google.common - - - com.google.common.**.testing.* - - - org.apache.beam.runners.direct.repackaged.com.google.common - - - - com.google.protobuf - - org.apache.beam.runners.direct.repackaged.com.google.protobuf - - - - com.google.thirdparty - - org.apache.beam.runners.direct.repackaged.com.google.thirdparty - - - - javax.annotation - - org.apache.beam.runners.direct.repackaged.javax.annotation - - - - - - - - - - - org.jacoco diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 9577ceb07cc3..c09b7f407a5b 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -142,6 +142,63 @@ true + + + org.apache.maven.plugins + maven-shade-plugin + + + bundle-and-repackage + package + + shade + + + true + + + com.google.guava:guava + org.apache.beam:beam-runners-core-construction-java + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + com.google.common + + + com.google.common.**.testing.* + + org.apache.beam.runners.dataflow.repackaged.com.google.common + + + com.google.thirdparty + org.apache.beam.runners.dataflow.repackaged.com.google.thirdparty + + + org.apache.beam.runners.core + org.apache.beam.runners.dataflow.repackaged.org.apache.beam.runners.core + + + + + + + + + @@ -173,70 +230,6 @@ - - - org.apache.maven.plugins - maven-jar-plugin - - - - org.apache.maven.plugins - maven-shade-plugin - - - bundle-and-repackage - package - - shade - - - true - - - com.google.guava:guava - org.apache.beam:beam-runners-core-construction-java - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - com.google.common - - - com.google.common.**.testing.* - - org.apache.beam.runners.dataflow.repackaged.com.google.common - - - com.google.thirdparty - org.apache.beam.runners.dataflow.repackaged.com.google.thirdparty - - - org.apache.beam.runners.core - org.apache.beam.runners.dataflow.repackaged.org.apache.beam.runners.core - - - - - - - - - - org.jacoco diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 882657b1db4e..ea273a2d2989 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -57,6 +57,85 @@ ${project.basedir}/src/test/ + + + org.apache.maven.plugins + maven-shade-plugin + + + bundle-and-repackage + package + + shade + + + true + + + com.google.guava:guava + com.google.protobuf:protobuf-java + net.bytebuddy:byte-buddy + org.apache.commons:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + + + com.google.common.**.testing.* + + + + org.apache.beam.sdk.repackaged.com.google.common + + + + com.google.thirdparty + + + org.apache.beam.sdk.repackaged.com.google.thirdparty + + + + com.google.protobuf + + + org.apache.beam.sdk.repackaged.com.google.protobuf + + + + net.bytebuddy + + + org.apache.beam.sdk.repackaged.net.bytebuddy + + + + org.apache.commons + + + org.apache.beam.sdk.repackaged.org.apache.commons + + + + + + + + + + @@ -120,92 +199,6 @@ - - - - org.apache.maven.plugins - maven-jar-plugin - - - - org.apache.maven.plugins - maven-shade-plugin - - - bundle-and-repackage - package - - shade - - - true - - - com.google.guava:guava - com.google.protobuf:protobuf-java - net.bytebuddy:byte-buddy - org.apache.commons:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - com.google.common - - - com.google.common.**.testing.* - - - - org.apache.beam.sdk.repackaged.com.google.common - - - - com.google.thirdparty - - - org.apache.beam.sdk.repackaged.com.google.thirdparty - - - - com.google.protobuf - - - org.apache.beam.sdk.repackaged.com.google.protobuf - - - - net.bytebuddy - - - org.apache.beam.sdk.repackaged.net.bytebuddy - - - - org.apache.commons - - - org.apache.beam.sdk.repackaged.org.apache.commons - - - - - - - - - - diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 7594365bc11c..ea2d8f0cb290 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -32,6 +32,21 @@ jar + + + + org.apache.maven.plugins + maven-shade-plugin + + + bundle-and-repackage + none + + + + + + org.apache.maven.plugins @@ -42,24 +57,6 @@ - - - - org.apache.maven.plugins - maven-jar-plugin - - - - org.apache.maven.plugins - maven-shade-plugin - - - bundle-and-repackage - none - - - From 3efb34b8322ac5933d9d71255d67e99d8e8361d7 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Thu, 18 May 2017 19:00:00 -0700 Subject: [PATCH 011/159] update dataflow python container --- .../python/apache_beam/runners/dataflow/internal/dependency.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/dependency.py b/sdks/python/apache_beam/runners/dataflow/internal/dependency.py index 892d9f94d1ee..e69c8d7dac50 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/dependency.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/dependency.py @@ -72,7 +72,8 @@ # Update this version to the next version whenever there is a change that will # require changes to the execution environment. -BEAM_CONTAINER_VERSION = '2.0.0' +# This should be in the beam-[version]-[date] format, date is optional. +BEAM_CONTAINER_VERSION = 'beam-2.1.0-20170518' # Standard file names used for staging files. WORKFLOW_TARBALL_FILE = 'workflow.tar.gz' From bac7bde5f878e6b2104c2e7cf3d01894a8f1b71d Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Wed, 17 May 2017 14:24:25 -0700 Subject: [PATCH 012/159] [BEAM-2164] Fix generate-sources.sh if directory contains space --- sdks/java/maven-archetypes/examples-java8/generate-sources.sh | 2 +- sdks/java/maven-archetypes/examples/generate-sources.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/maven-archetypes/examples-java8/generate-sources.sh b/sdks/java/maven-archetypes/examples-java8/generate-sources.sh index 58c5f22298b7..d8117ce5f266 100755 --- a/sdks/java/maven-archetypes/examples-java8/generate-sources.sh +++ b/sdks/java/maven-archetypes/examples-java8/generate-sources.sh @@ -22,7 +22,7 @@ # Usage: Invoke with no arguments from any working directory. # The directory of this script. Assumes root of the maven-archetypes module. -HERE="$(dirname $0)" +HERE="$( dirname "$0" )" # The directory of the examples-java and examples-java8 modules EXAMPLES_ROOT="${HERE}/../../../../examples/java" diff --git a/sdks/java/maven-archetypes/examples/generate-sources.sh b/sdks/java/maven-archetypes/examples/generate-sources.sh index d9109ac313c2..62af772884f5 100755 --- a/sdks/java/maven-archetypes/examples/generate-sources.sh +++ b/sdks/java/maven-archetypes/examples/generate-sources.sh @@ -21,7 +21,7 @@ # Usage: Invoke with no arguments from any working directory. # The directory of this script. Assumes root of the maven-archetypes module. -HERE="$(dirname $0)" +HERE="$( dirname "$0" )" # The directory of the examples-java module EXAMPLES_ROOT="${HERE}/../../../../examples/java" From c7bd2b43ddf726248291a653f8d7d796a51800a0 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 19 May 2017 11:40:20 -0700 Subject: [PATCH 013/159] Move snappy-java version to dependencyManagement --- pom.xml | 7 +++++++ sdks/java/core/pom.xml | 1 - 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a978f58fe3b1..ed1a076b8356 100644 --- a/pom.xml +++ b/pom.xml @@ -141,6 +141,7 @@ 4.4.1 4.3.5.RELEASE 2.0 + 1.1.4-M3 2.20 2.20 3.0.2 @@ -993,6 +994,12 @@ ${spring.version} + + org.xerial.snappy + snappy-java + ${snappy-java.version} + + diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 882657b1db4e..c5a81c012558 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -266,7 +266,6 @@ org.xerial.snappy snappy-java - 1.1.4-M3 From c4d14f8be7bb72fd653f1ab8e8080fc2b65f6672 Mon Sep 17 00:00:00 2001 From: Colin Phipps Date: Tue, 25 Apr 2017 15:28:28 +0000 Subject: [PATCH 014/159] Comply with byte limit for Datastore Commit. --- .../sdk/io/gcp/datastore/DatastoreV1.java | 16 +++++++++ .../sdk/io/gcp/datastore/DatastoreV1Test.java | 34 +++++++++++++++++++ 2 files changed, 50 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index 16bb1b4d36a1..4cfb80119559 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -207,6 +207,13 @@ public class DatastoreV1 { @VisibleForTesting static final int DATASTORE_BATCH_UPDATE_LIMIT = 500; + /** + * Cloud Datastore has a limit of 10MB per RPC, so we also flush if the total size of mutations + * exceeds this limit. This is set lower than the 10MB limit on the RPC, as this only accounts for + * the mutations themselves and not the CommitRequest wrapper around them. + */ + static final int DATASTORE_BATCH_UPDATE_BYTES_LIMIT = 5_000_000; + /** * Returns an empty {@link DatastoreV1.Read} builder. Configure the source {@code projectId}, * {@code query}, and optionally {@code namespace} and {@code numQuerySplits} using @@ -1123,6 +1130,7 @@ static class DatastoreWriterFn extends DoFn { private final V1DatastoreFactory datastoreFactory; // Current batch of mutations to be written. private final List mutations = new ArrayList<>(); + private int mutationsSize = 0; // Accumulated size of protos in mutations. private static final int MAX_RETRIES = 5; private static final FluentBackoff BUNDLE_WRITE_BACKOFF = @@ -1152,7 +1160,14 @@ public void startBundle(StartBundleContext c) { @ProcessElement public void processElement(ProcessContext c) throws Exception { + Mutation write = c.element(); + int size = write.getSerializedSize(); + if (mutations.size() > 0 + && mutationsSize + size >= DatastoreV1.DATASTORE_BATCH_UPDATE_BYTES_LIMIT) { + flushBatch(); + } mutations.add(c.element()); + mutationsSize += size; if (mutations.size() >= DatastoreV1.DATASTORE_BATCH_UPDATE_LIMIT) { flushBatch(); } @@ -1203,6 +1218,7 @@ private void flushBatch() throws DatastoreException, IOException, InterruptedExc } LOG.debug("Successfully wrote {} mutations", mutations.size()); mutations.clear(); + mutationsSize = 0; } @Override diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index ba8ac846ffc7..3597b54fa16a 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -60,6 +60,7 @@ import com.google.datastore.v1.QueryResultBatch; import com.google.datastore.v1.RunQueryRequest; import com.google.datastore.v1.RunQueryResponse; +import com.google.datastore.v1.Value; import com.google.datastore.v1.client.Datastore; import com.google.datastore.v1.client.DatastoreException; import com.google.datastore.v1.client.QuerySplitter; @@ -644,6 +645,39 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { } } + /** + * Tests {@link DatastoreWriterFn} with large entities that need to be split into more batches. + */ + @Test + public void testDatatoreWriterFnWithLargeEntities() throws Exception { + List mutations = new ArrayList<>(); + for (int i = 0; i < 12; ++i) { + Entity.Builder entity = Entity.newBuilder().setKey(makeKey("key" + i, i + 1)); + entity.putProperties("long", Value.newBuilder().setStringValue(new String(new char[1_000_000]) + ).setExcludeFromIndexes(true).build()); + mutations.add(makeUpsert(entity.build()).build()); + } + + DatastoreWriterFn datastoreWriter = new DatastoreWriterFn(StaticValueProvider.of(PROJECT_ID), + null, mockDatastoreFactory); + DoFnTester doFnTester = DoFnTester.of(datastoreWriter); + doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); + doFnTester.processBundle(mutations); + + // This test is over-specific currently; it requires that we split the 12 entity writes into 3 + // requests, but we only need each CommitRequest to be less than 10MB in size. + int start = 0; + while (start < mutations.size()) { + int end = Math.min(mutations.size(), start + 4); + CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + commitRequest.addAllMutations(mutations.subList(start, end)); + // Verify all the batch requests were made with the expected mutations. + verify(mockDatastore).commit(commitRequest.build()); + start = end; + } + } + /** * Tests {@link DatastoreV1.Read#getEstimatedSizeBytes} to fetch and return estimated size for a * query. From de95c7f5bf3b8391486b898b0ffb1a0a05338725 Mon Sep 17 00:00:00 2001 From: Colin Phipps Date: Tue, 9 May 2017 15:07:51 +0000 Subject: [PATCH 015/159] Make the large entities test neater. --- .../org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java | 1 + .../apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java | 8 +++++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index 4cfb80119559..b198a6f568b9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -212,6 +212,7 @@ public class DatastoreV1 { * exceeds this limit. This is set lower than the 10MB limit on the RPC, as this only accounts for * the mutations themselves and not the CommitRequest wrapper around them. */ + @VisibleForTesting static final int DATASTORE_BATCH_UPDATE_BYTES_LIMIT = 5_000_000; /** diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index 3597b54fa16a..460049e8b355 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -26,6 +26,7 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeOrder; import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DATASTORE_BATCH_UPDATE_BYTES_LIMIT; import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DATASTORE_BATCH_UPDATE_LIMIT; import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.DEFAULT_BUNDLE_SIZE_BYTES; import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.QUERY_BATCH_LIMIT; @@ -60,7 +61,6 @@ import com.google.datastore.v1.QueryResultBatch; import com.google.datastore.v1.RunQueryRequest; import com.google.datastore.v1.RunQueryResponse; -import com.google.datastore.v1.Value; import com.google.datastore.v1.client.Datastore; import com.google.datastore.v1.client.DatastoreException; import com.google.datastore.v1.client.QuerySplitter; @@ -651,9 +651,10 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { @Test public void testDatatoreWriterFnWithLargeEntities() throws Exception { List mutations = new ArrayList<>(); + int propertySize = 900_000; for (int i = 0; i < 12; ++i) { Entity.Builder entity = Entity.newBuilder().setKey(makeKey("key" + i, i + 1)); - entity.putProperties("long", Value.newBuilder().setStringValue(new String(new char[1_000_000]) + entity.putProperties("long", makeValue(new String(new char[propertySize]) ).setExcludeFromIndexes(true).build()); mutations.add(makeUpsert(entity.build()).build()); } @@ -666,9 +667,10 @@ public void testDatatoreWriterFnWithLargeEntities() throws Exception { // This test is over-specific currently; it requires that we split the 12 entity writes into 3 // requests, but we only need each CommitRequest to be less than 10MB in size. + int propertiesPerRpc = DATASTORE_BATCH_UPDATE_BYTES_LIMIT / propertySize; int start = 0; while (start < mutations.size()) { - int end = Math.min(mutations.size(), start + 4); + int end = Math.min(mutations.size(), start + propertiesPerRpc); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); commitRequest.addAllMutations(mutations.subList(start, end)); From dcf405644e2b51303e1d2c12592fe82ee01eb32f Mon Sep 17 00:00:00 2001 From: Colin Phipps Date: Tue, 9 May 2017 09:40:50 +0000 Subject: [PATCH 016/159] End-to-end test for large entity writes. --- .../beam/sdk/io/gcp/datastore/V1ReadIT.java | 2 +- .../beam/sdk/io/gcp/datastore/V1TestUtil.java | 15 ++++++-- .../beam/sdk/io/gcp/datastore/V1WriteIT.java | 36 +++++++++++++++++-- 3 files changed, 47 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java index ec7fa8f046d9..22945f599ce5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java @@ -148,7 +148,7 @@ private static void writeEntitiesToDatastore(V1TestOptions options, String proje Key ancestorKey = makeAncestorKey(options.getNamespace(), options.getKind(), ancestor); for (long i = 0; i < numEntities; i++) { - Entity entity = makeEntity(i, ancestorKey, options.getKind(), options.getNamespace()); + Entity entity = makeEntity(i, ancestorKey, options.getKind(), options.getNamespace(), 0); writer.write(entity); } writer.close(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index dc91638878dd..5e618dfaabd2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -92,8 +92,10 @@ static Query makeAncestorKindQuery(String kind, @Nullable String namespace, Stri /** * Build an entity for the given ancestorKey, kind, namespace and value. + * @param largePropertySize if greater than 0, add an unindexed property of the given size. */ - static Entity makeEntity(Long value, Key ancestorKey, String kind, @Nullable String namespace) { + static Entity makeEntity(Long value, Key ancestorKey, String kind, @Nullable String namespace, + int largePropertySize) { Entity.Builder entityBuilder = Entity.newBuilder(); Key.Builder keyBuilder = makeKey(ancestorKey, kind, UUID.randomUUID().toString()); // NOTE: Namespace is not inherited between keys created with DatastoreHelper.makeKey, so @@ -105,6 +107,10 @@ static Entity makeEntity(Long value, Key ancestorKey, String kind, @Nullable Str entityBuilder.setKey(keyBuilder.build()); entityBuilder.putProperties("value", makeValue(value).build()); + if (largePropertySize > 0) { + entityBuilder.putProperties("unindexed_value", makeValue(new String( + new char[largePropertySize]).replace("\0", "A")).setExcludeFromIndexes(true).build()); + } return entityBuilder.build(); } @@ -115,18 +121,21 @@ static class CreateEntityFn extends DoFn { private final String kind; @Nullable private final String namespace; + private final int largePropertySize; private Key ancestorKey; - CreateEntityFn(String kind, @Nullable String namespace, String ancestor) { + CreateEntityFn(String kind, @Nullable String namespace, String ancestor, + int largePropertySize) { this.kind = kind; this.namespace = namespace; + this.largePropertySize = largePropertySize; // Build the ancestor key for all created entities once, including the namespace. ancestorKey = makeAncestorKey(namespace, kind, ancestor); } @ProcessElement public void processElement(ProcessContext c) throws Exception { - c.output(makeEntity(c.element(), ancestorKey, kind, namespace)); + c.output(makeEntity(c.element(), ancestorKey, kind, namespace, largePropertySize)); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java index 82e4d6430ab9..4a874fd7ef4d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java @@ -67,8 +67,7 @@ public void testE2EV1Write() throws Exception { // Write to datastore p.apply(GenerateSequence.from(0).to(numEntities)) - .apply(ParDo.of(new CreateEntityFn( - options.getKind(), options.getNamespace(), ancestor))) + .apply(ParDo.of(new CreateEntityFn(options.getKind(), options.getNamespace(), ancestor, 0))) .apply(DatastoreIO.v1().write().withProjectId(project)); p.run(); @@ -79,6 +78,39 @@ public void testE2EV1Write() throws Exception { assertEquals(numEntitiesWritten, numEntities); } + /** + * An end-to-end test for {@link DatastoreV1.Write}. + * + *

Write some large test entities to Cloud Datastore, to test that a batch is flushed when + * the byte size limit is reached. Read and count all the entities. Verify that the count matches + * the number of entities written. + */ + @Test + public void testE2EV1WriteWithLargeEntities() throws Exception { + Pipeline p = Pipeline.create(options); + + /* + * Datastore has a limit of 1MB per entity, and 10MB per write RPC. If each entity is around + * 1MB in size, then we hit the limit on the size of the write long before we hit the limit on + * the number of entities per writes. + */ + final int rawPropertySize = 900_000; + final int numLargeEntities = 100; + + // Write to datastore + p.apply(GenerateSequence.from(0).to(numLargeEntities)) + .apply(ParDo.of(new CreateEntityFn( + options.getKind(), options.getNamespace(), ancestor, rawPropertySize))) + .apply(DatastoreIO.v1().write().withProjectId(project)); + + p.run(); + + // Count number of entities written to datastore. + long numEntitiesWritten = countEntities(options, project, ancestor); + + assertEquals(numEntitiesWritten, numLargeEntities); + } + @After public void tearDown() throws Exception { deleteAllEntities(options, project, ancestor); From 383329d6c35e50c8d907ac97522a64236bb98dd0 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Fri, 12 May 2017 16:44:10 -0700 Subject: [PATCH 017/159] [BEAM-1345] Remove FileSystems.setDefaultConfigInWorkers since Dataflow no longer depends on this --- runners/google-cloud-dataflow-java/pom.xml | 2 +- .../src/main/java/org/apache/beam/sdk/io/FileSystems.java | 6 ------ 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index c09b7f407a5b..895a8e6a513b 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -33,7 +33,7 @@ jar - beam-master-20170512 + beam-master-20170519 1 6 diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java index 1aacc9020705..2ed29e3a6718 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java @@ -449,12 +449,6 @@ static FileSystem getFileSystemInternal(String scheme) { /********************************** METHODS FOR REGISTRATION **********************************/ - /** @deprecated to be removed. */ - @Deprecated // for DataflowRunner backwards compatibility. - public static void setDefaultConfigInWorkers(PipelineOptions options) { - setDefaultPipelineOptions(options); - } - /** * Sets the default configuration in workers. * From 6c6d7877620af5265ed5ca14dd2ed7cd905b7404 Mon Sep 17 00:00:00 2001 From: "wtanaka.com" Date: Thu, 18 May 2017 20:42:46 -1000 Subject: [PATCH 018/159] [BEAM-2293] Rename Top.{Largest,Smallest=>Natural,Reversed} - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [x] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf). --- --- .../sdk/transforms/ApproximateQuantiles.java | 4 +-- .../org/apache/beam/sdk/transforms/Max.java | 4 +-- .../org/apache/beam/sdk/transforms/Min.java | 4 +-- .../org/apache/beam/sdk/transforms/Top.java | 36 +++++++++++++++---- .../transforms/ApproximateQuantilesTest.java | 2 +- .../apache/beam/sdk/transforms/MaxTest.java | 2 +- .../apache/beam/sdk/transforms/MinTest.java | 2 +- .../apache/beam/sdk/transforms/TopTest.java | 10 +++--- 8 files changed, 44 insertions(+), 20 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java index d12d193edc00..e952af2bff7f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java @@ -283,8 +283,8 @@ ApproximateQuantilesCombineFn create( * Like {@link #create(int, Comparator)}, but sorts values using their natural ordering. */ public static > - ApproximateQuantilesCombineFn> create(int numQuantiles) { - return create(numQuantiles, new Top.Largest()); + ApproximateQuantilesCombineFn> create(int numQuantiles) { + return create(numQuantiles, new Top.Natural()); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java index 91851bcbe9f6..710fe77a0d70 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java @@ -158,11 +158,11 @@ BinaryCombineFn of(final ComparatorT comparator) { } public static > BinaryCombineFn naturalOrder(T identity) { - return new MaxFn(identity, new Top.Largest()); + return new MaxFn(identity, new Top.Natural()); } public static > BinaryCombineFn naturalOrder() { - return new MaxFn(null, new Top.Largest()); + return new MaxFn(null, new Top.Natural()); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java index 109f4e5cb5bc..c566fb3e6ff2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java @@ -158,11 +158,11 @@ BinaryCombineFn of(ComparatorT comparator) { } public static > BinaryCombineFn naturalOrder(T identity) { - return new MinFn(identity, new Top.Largest()); + return new MinFn(identity, new Top.Natural()); } public static > BinaryCombineFn naturalOrder() { - return new MinFn(null, new Top.Largest()); + return new MinFn(null, new Top.Natural()); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java index 99ec49bb8ef8..319f7798e6d1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java @@ -144,7 +144,7 @@ Combine.Globally> of(int count, ComparatorT compareFn) { * {@code KV}s and return the top values associated with each key. */ public static > Combine.Globally> smallest(int count) { - return Combine.globally(new TopCombineFn<>(count, new Smallest())); + return Combine.globally(new TopCombineFn<>(count, new Reversed())); } /** @@ -188,7 +188,7 @@ public static > Combine.Globally> smallest(in * {@code KV}s and return the top values associated with each key. */ public static > Combine.Globally> largest(int count) { - return Combine.globally(new TopCombineFn<>(count, new Largest())); + return Combine.globally(new TopCombineFn<>(count, new Natural())); } /** @@ -281,7 +281,7 @@ public static > Combine.Globally> largest(int public static > PTransform>, PCollection>>> smallestPerKey(int count) { - return Combine.perKey(new TopCombineFn<>(count, new Smallest())); + return Combine.perKey(new TopCombineFn<>(count, new Reversed())); } /** @@ -327,14 +327,26 @@ public static > Combine.Globally> largest(int public static > PerKey> largestPerKey(int count) { - return Combine.perKey(new TopCombineFn<>(count, new Largest())); + return Combine.perKey(new TopCombineFn<>(count, new Natural())); + } + + /** + * @deprecated use {@link Natural} instead + */ + @Deprecated + public static class Largest> + implements Comparator, Serializable { + @Override + public int compare(T a, T b) { + return a.compareTo(b); + } } /** * A {@code Serializable} {@code Comparator} that that uses the compared elements' natural * ordering. */ - public static class Largest> + public static class Natural> implements Comparator, Serializable { @Override public int compare(T a, T b) { @@ -342,11 +354,23 @@ public int compare(T a, T b) { } } + /** + * @deprecated use {@link Reversed} instead + */ + @Deprecated + public static class Smallest> + implements Comparator, Serializable { + @Override + public int compare(T a, T b) { + return b.compareTo(a); + } + } + /** * {@code Serializable} {@code Comparator} that that uses the reverse of the compared elements' * natural ordering. */ - public static class Smallest> + public static class Reversed> implements Comparator, Serializable { @Override public int compare(T a, T b) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java index cd7898bc95a6..9e0b3cc9069e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateQuantilesTest.java @@ -236,7 +236,7 @@ public void testAlternateComparator() { @Test public void testDisplayData() { - Top.Largest comparer = new Top.Largest(); + Top.Natural comparer = new Top.Natural(); PTransform approxQuanitiles = ApproximateQuantiles.globally(20, comparer); DisplayData displayData = DisplayData.from(approxQuanitiles); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java index 2b4356094894..52043e18597c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java @@ -69,7 +69,7 @@ public void testMaxDoubleFn() { @Test public void testDisplayData() { - Top.Largest comparer = new Top.Largest<>(); + Top.Natural comparer = new Top.Natural<>(); Combine.Globally max = Max.globally(comparer); assertThat(DisplayData.from(max), hasDisplayItem("comparer", comparer.getClass())); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java index e89b2235b221..1ece09b532f7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java @@ -69,7 +69,7 @@ public void testMinDoubleFn() { @Test public void testDisplayData() { - Top.Smallest comparer = new Top.Smallest<>(); + Top.Reversed comparer = new Top.Reversed<>(); Combine.Globally min = Min.globally(comparer); assertThat(DisplayData.from(min), hasDisplayItem("comparer", comparer.getClass())); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java index 9b0b27d0adbc..a7f439c2c2ee 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/TopTest.java @@ -231,17 +231,17 @@ public void testCountConstraint() { @Test public void testTopGetNames() { assertEquals("Combine.globally(Top(OrderByLength))", Top.of(1, new OrderByLength()).getName()); - assertEquals("Combine.globally(Top(Smallest))", Top.smallest(1).getName()); - assertEquals("Combine.globally(Top(Largest))", Top.largest(2).getName()); + assertEquals("Combine.globally(Top(Reversed))", Top.smallest(1).getName()); + assertEquals("Combine.globally(Top(Natural))", Top.largest(2).getName()); assertEquals("Combine.perKey(Top(IntegerComparator))", Top.perKey(1, new IntegerComparator()).getName()); - assertEquals("Combine.perKey(Top(Smallest))", Top.smallestPerKey(1).getName()); - assertEquals("Combine.perKey(Top(Largest))", Top.largestPerKey(2).getName()); + assertEquals("Combine.perKey(Top(Reversed))", Top.smallestPerKey(1).getName()); + assertEquals("Combine.perKey(Top(Natural))", Top.largestPerKey(2).getName()); } @Test public void testDisplayData() { - Top.Largest comparer = new Top.Largest(); + Top.Natural comparer = new Top.Natural(); Combine.Globally> top = Top.of(1234, comparer); DisplayData displayData = DisplayData.from(top); From 2ed68f7f49a6f84921b0de03a57b0bbe61d263a7 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Mon, 15 May 2017 15:43:00 -0700 Subject: [PATCH 019/159] [BEAM-2253] Use resource transformer plugin to use versions defined in root pom.xml within maven archetype poms --- pom.xml | 12 +++-- .../resources/archetype-resources/pom.xml | 51 ++++++++++++------- .../resources/archetype-resources/pom.xml | 48 +++++++++++------ .../resources/archetype-resources/pom.xml | 12 +++-- .../projects/basic/reference/pom.xml | 12 +++-- 5 files changed, 90 insertions(+), 45 deletions(-) diff --git a/pom.xml b/pom.xml index a978f58fe3b1..211a8221290d 100644 --- a/pom.xml +++ b/pom.xml @@ -143,8 +143,12 @@ 2.0 2.20 2.20 + 3.6.1 + 1.4.0 + 3.0.2 3.0.2 - + 3.0.0 + -Werror -Xpkginfo:always nothing @@ -1131,7 +1135,7 @@ maven-compiler-plugin - 3.6.1 + ${maven-compiler-plugin.version} 1.7 1.7 @@ -1186,7 +1190,7 @@ org.apache.maven.plugins maven-jar-plugin - 3.0.2 + ${maven-jar-plugin.version} default-jar @@ -1418,7 +1422,7 @@ org.apache.maven.plugins maven-shade-plugin - 3.0.0 + ${maven-shade-plugin.version} bundle-and-repackage diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml index 5f34689f028f..47dfc91face3 100644 --- a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml @@ -28,7 +28,23 @@ @project.version@ - 2.20 + + @bigquery.version@ + @google-clients.version@ + @guava.version@ + @hamcrest.version@ + @jackson.version@ + @joda.version@ + @junit.version@ + @maven-compiler-plugin.version@ + @maven-exec-plugin.version@ + @maven-jar-plugin.version@ + @maven-shade-plugin.version@ + @mockito.version@ + @pubsub.version@ + @slf4j.version@ + @spark.version@ + @surefire-plugin.version@ @@ -50,7 +66,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.5.1 + ${maven-compiler-plugin.version} 1.8 1.8 @@ -80,6 +96,7 @@ org.apache.maven.plugins maven-jar-plugin + ${maven-jar-plugin.version} @@ -276,7 +293,7 @@ com.google.apis google-api-services-bigquery - v2-rev295-1.22.0 + ${bigquery.version} @@ -290,7 +307,7 @@ com.google.http-client google-http-client - 1.22.0 + ${google-clients.version} @@ -304,7 +321,7 @@ com.google.apis google-api-services-pubsub - v1-rev10-1.22.0 + ${pubsub.version} @@ -318,26 +335,26 @@ joda-time joda-time - 2.4 + ${joda.version} com.google.guava guava - 20.0 + ${guava.version} org.slf4j slf4j-api - 1.7.14 + ${slf4j.version} org.slf4j slf4j-jdk14 - 1.7.14 + ${slf4j.version} runtime @@ -347,19 +364,19 @@ org.hamcrest hamcrest-all - 1.3 + ${hamcrest.version} junit junit - 4.12 + ${junit.version} org.mockito mockito-all - 1.9.5 + ${mockito.version} test diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index a3d7b8fd73b6..025e3565a21c 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -28,7 +28,22 @@ @project.version@ - 2.20 + + @bigquery.version@ + @google-clients.version@ + @guava.version@ + @hamcrest.version@ + @jackson.version@ + @joda.version@ + @junit.version@ + @maven-compiler-plugin.version@ + @maven-exec-plugin.version@ + @maven-jar-plugin.version@ + @maven-shade-plugin.version@ + @pubsub.version@ + @slf4j.version@ + @spark.version@ + @surefire-plugin.version@ @@ -50,7 +65,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.5.1 + ${maven-compiler-plugin.version} ${targetPlatform} ${targetPlatform} @@ -80,6 +95,7 @@ org.apache.maven.plugins maven-jar-plugin + ${maven-jar-plugin.version} @@ -276,7 +292,7 @@ com.google.apis google-api-services-bigquery - v2-rev295-1.22.0 + ${bigquery.version} @@ -290,7 +306,7 @@ com.google.http-client google-http-client - 1.22.0 + ${google-clients.version} @@ -304,7 +320,7 @@ com.google.apis google-api-services-pubsub - v1-rev10-1.22.0 + ${pubsub.version} @@ -318,26 +334,26 @@ joda-time joda-time - 2.4 + ${joda.version} com.google.guava guava - 20.0 + ${guava.version} org.slf4j slf4j-api - 1.7.14 + ${slf4j.version} org.slf4j slf4j-jdk14 - 1.7.14 + ${slf4j.version} runtime @@ -347,13 +363,13 @@ org.hamcrest hamcrest-all - 1.3 + ${hamcrest.version} junit junit - 4.12 + ${junit.version} diff --git a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml index dddd5ab8a819..f4fb9f8df26f 100644 --- a/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/starter/src/main/resources/archetype-resources/pom.xml @@ -26,6 +26,10 @@ @project.version@ + + @maven-compiler-plugin.version@ + @maven-exec-plugin.version@ + @slf4j.version@ @@ -47,7 +51,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.5.1 + ${maven-compiler-plugin.version} ${targetPlatform} ${targetPlatform} @@ -60,7 +64,7 @@ org.codehaus.mojo exec-maven-plugin - 1.4.0 + ${maven-exec-plugin.version} false @@ -94,12 +98,12 @@ org.slf4j slf4j-api - 1.7.14 + ${slf4j.version} org.slf4j slf4j-jdk14 - 1.7.14 + ${slf4j.version} diff --git a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml index 39fefd6ff60a..60405e6dbb17 100644 --- a/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml +++ b/sdks/java/maven-archetypes/starter/src/test/resources/projects/basic/reference/pom.xml @@ -26,6 +26,10 @@ @project.version@ + + 3.6.1 + 1.4.0 + 1.7.14 @@ -47,7 +51,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.5.1 + ${maven-compiler-plugin.version} 1.7 1.7 @@ -60,7 +64,7 @@ org.codehaus.mojo exec-maven-plugin - 1.4.0 + ${maven-exec-plugin.version} false @@ -94,12 +98,12 @@ org.slf4j slf4j-api - 1.7.14 + ${slf4j.version} org.slf4j slf4j-jdk14 - 1.7.14 + ${slf4j.version} From 5c163c77bd6d948231ea5a5ce9cce7a1064b0331 Mon Sep 17 00:00:00 2001 From: Romain Yon Date: Fri, 19 May 2017 16:16:24 -0400 Subject: [PATCH 020/159] [BEAM-2334] Fix OutOfMemoryError --- .../org/apache/beam/runners/dataflow/util/RandomAccessData.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java index 5ea9f07ff6a9..0c0890243d25 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java @@ -350,7 +350,7 @@ private void ensureCapacity(int minCapacity) { // Try to double the size of the buffer, if thats not enough, just use the new capacity. // Note that we use Math.min(long, long) to not cause overflow on the multiplication. - int newCapacity = (int) Math.min(Integer.MAX_VALUE, buffer.length * 2L); + int newCapacity = (int) Math.min(Integer.MAX_VALUE - 8, buffer.length * 2L); if (newCapacity < minCapacity) { newCapacity = minCapacity; } From 4a63970261be22660f5fef8bf37b5d18301315ef Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 19 May 2017 15:24:19 -0700 Subject: [PATCH 021/159] Add TransformPayloadTranslatorRegistrar --- runners/core-construction-java/pom.xml | 7 ++++- .../core/construction/PTransforms.java | 21 +++++++++----- .../runners/core/construction/ParDos.java | 15 ++++++++++ .../TransformPayloadTranslatorRegistrar.java | 29 +++++++++++++++++++ 4 files changed, 64 insertions(+), 8 deletions(-) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java diff --git a/runners/core-construction-java/pom.xml b/runners/core-construction-java/pom.xml index abf0b6512fec..7eaa6f35b2f9 100644 --- a/runners/core-construction-java/pom.xml +++ b/runners/core-construction-java/pom.xml @@ -89,6 +89,12 @@ slf4j-api + + com.google.auto.service + auto-service + true + + com.google.auto.value auto-value @@ -114,6 +120,5 @@ mockito-all test - diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java index 16276b9ed82c..9826b77b300a 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java @@ -24,12 +24,11 @@ import java.io.IOException; import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.construction.ParDos.ParDoPayloadTranslator; +import java.util.ServiceLoader; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; @@ -40,11 +39,19 @@ */ public class PTransforms { private static final Map, TransformPayloadTranslator> - KNOWN_PAYLOAD_TRANSLATORS = - ImmutableMap., TransformPayloadTranslator>builder() - .put(ParDo.MultiOutput.class, ParDoPayloadTranslator.create()) - .build(); - // TODO: Load via service loader. + KNOWN_PAYLOAD_TRANSLATORS = loadTransformPayloadTranslators(); + + private static Map, TransformPayloadTranslator> + loadTransformPayloadTranslators() { + ImmutableMap.Builder, TransformPayloadTranslator> builder = + ImmutableMap.builder(); + for (TransformPayloadTranslatorRegistrar registrar : + ServiceLoader.load(TransformPayloadTranslatorRegistrar.class)) { + builder.putAll(registrar.getTransformPayloadTranslators()); + } + return builder.build(); + } + private PTransforms() {} /** diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java index b2b29df65dc0..4752bd1a8940 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; import com.google.common.base.Optional; import com.google.protobuf.Any; @@ -28,6 +29,7 @@ import com.google.protobuf.InvalidProtocolBufferException; import java.io.IOException; import java.io.Serializable; +import java.util.Collections; import java.util.List; import java.util.Map; import org.apache.beam.runners.core.construction.PTransforms.TransformPayloadTranslator; @@ -46,6 +48,7 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Materializations; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.ParDo.MultiOutput; import org.apache.beam.sdk.transforms.ViewFn; @@ -107,6 +110,18 @@ public FunctionSpec translate( .setParameter(Any.pack(payload)) .build(); } + + /** + * Registers {@link ParDoPayloadTranslator}. + */ + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class Registrar implements TransformPayloadTranslatorRegistrar { + @Override + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return Collections.singletonMap(ParDo.MultiOutput.class, new ParDoPayloadTranslator()); + } + } } public static ParDoPayload toProto(ParDo.MultiOutput parDo, SdkComponents components) { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java new file mode 100644 index 000000000000..bc568a616159 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import java.util.Map; +import org.apache.beam.runners.core.construction.PTransforms.TransformPayloadTranslator; +import org.apache.beam.sdk.transforms.PTransform; + +/** A registrar of TransformPayloadTranslator. */ +public interface TransformPayloadTranslatorRegistrar { + Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators(); +} From 5e9fcebc07725de368391914781e5b4d5f9c4a19 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 19 May 2017 12:57:41 -0700 Subject: [PATCH 022/159] Remove Pipeline reference from TransformHierarchy This change removes a direct dependency cycle between Pipeline and TransformHierarchy. There is still an indirect cycle through PValues, but that is slightly less problematic. --- .../translation/ApexPipelineTranslator.java | 4 +- .../apex/translation/TranslationContext.java | 5 +- .../core/construction/SdkComponents.java | 14 +++--- .../core/construction/SdkComponentsTest.java | 7 +-- .../runners/direct/DirectGraphVisitor.java | 3 +- .../direct/KeyedPValueTrackingVisitor.java | 5 +- .../flink/FlinkBatchPipelineTranslator.java | 2 +- .../beam/runners/flink/FlinkRunner.java | 10 +--- .../FlinkStreamingPipelineTranslator.java | 4 +- .../dataflow/DataflowPipelineTranslator.java | 6 +-- .../beam/runners/dataflow/DataflowRunner.java | 2 +- .../beam/runners/spark/SparkRunner.java | 2 +- .../streaming/TrackStreamingSourcesTest.java | 14 +++++- .../java/org/apache/beam/sdk/Pipeline.java | 50 +++++++++++++++---- .../beam/sdk/runners/TransformHierarchy.java | 6 +-- .../sdk/runners/TransformHierarchyTest.java | 2 +- 16 files changed, 82 insertions(+), 54 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java index 32e470f6029a..bda074b0a29b 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ApexPipelineTranslator.java @@ -49,7 +49,7 @@ * into Apex logical plan {@link DAG}. */ @SuppressWarnings({"rawtypes", "unchecked"}) -public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { +public class ApexPipelineTranslator extends Pipeline.PipelineVisitor.Defaults { private static final Logger LOG = LoggerFactory.getLogger(ApexPipelineTranslator.class); /** @@ -110,7 +110,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { throw new UnsupportedOperationException( "no translator registered for " + transform); } - translationContext.setCurrentTransform(node); + translationContext.setCurrentTransform(node.toAppliedPTransform(getPipeline())); translator.translate(transform, translationContext); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java index a5e30281a9e3..aff3863624c4 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java @@ -36,7 +36,6 @@ import org.apache.beam.runners.apex.translation.utils.CoderAdapterStreamCodec; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.Window; @@ -77,8 +76,8 @@ public InputT getViewInput(PCollectionView view) { this.pipelineOptions = pipelineOptions; } - public void setCurrentTransform(TransformHierarchy.Node treeNode) { - this.currentTransform = treeNode.toAppliedPTransform(); + public void setCurrentTransform(AppliedPTransform transform) { + this.currentTransform = transform; } public ApexPipelineOptions getPipelineOptions() { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java index eb29b9a3ae7a..5714fc510481 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java @@ -62,10 +62,10 @@ static SdkComponents create() { return new SdkComponents(); } - public static RunnerApi.Pipeline translatePipeline(Pipeline p) { + public static RunnerApi.Pipeline translatePipeline(Pipeline pipeline) { final SdkComponents components = create(); final Collection rootIds = new HashSet<>(); - p.traverseTopologically( + pipeline.traverseTopologically( new PipelineVisitor.Defaults() { private final ListMultimap> children = ArrayListMultimap.create(); @@ -77,9 +77,10 @@ public void leaveCompositeTransform(Node node) { rootIds.add(components.getExistingPTransformId(pipelineRoot)); } } else { - children.put(node.getEnclosingNode(), node.toAppliedPTransform()); + children.put(node.getEnclosingNode(), node.toAppliedPTransform(getPipeline())); try { - components.registerPTransform(node.toAppliedPTransform(), children.get(node)); + components.registerPTransform( + node.toAppliedPTransform(getPipeline()), children.get(node)); } catch (IOException e) { throw new RuntimeException(e); } @@ -88,10 +89,11 @@ public void leaveCompositeTransform(Node node) { @Override public void visitPrimitiveTransform(Node node) { - children.put(node.getEnclosingNode(), node.toAppliedPTransform()); + children.put(node.getEnclosingNode(), node.toAppliedPTransform(getPipeline())); try { components.registerPTransform( - node.toAppliedPTransform(), Collections.>emptyList()); + node.toAppliedPTransform(getPipeline()), + Collections.>emptyList()); } catch (IOException e) { throw new IllegalStateException(e); } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java index 7424886d1009..55702ea22390 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java @@ -97,17 +97,12 @@ public void translatePipeline() { final RunnerApi.Pipeline pipelineProto = SdkComponents.translatePipeline(pipeline); pipeline.traverseTopologically( - new PipelineVisitor() { + new PipelineVisitor.Defaults() { Set transforms = new HashSet<>(); Set> pcollections = new HashSet<>(); Set>> coders = new HashSet<>(); Set> windowingStrategies = new HashSet<>(); - @Override - public CompositeBehavior enterCompositeTransform(Node node) { - return CompositeBehavior.ENTER_TRANSFORM; - } - @Override public void leaveCompositeTransform(Node node) { if (node.isRootNode()) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index 1ee8ceb9a7e5..01204e3049dd 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -41,6 +41,7 @@ * input after the upstream transform has produced and committed output. */ class DirectGraphVisitor extends PipelineVisitor.Defaults { + private Map> producers = new HashMap<>(); private ListMultimap> primitiveConsumers = @@ -101,7 +102,7 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { private AppliedPTransform getAppliedTransform(TransformHierarchy.Node node) { @SuppressWarnings({"rawtypes", "unchecked"}) - AppliedPTransform application = node.toAppliedPTransform(); + AppliedPTransform application = node.toAppliedPTransform(getPipeline()); return application; } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java index 347f313af85a..f9b2daeaa1df 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java @@ -44,7 +44,7 @@ */ // TODO: Handle Key-preserving transforms when appropriate and more aggressively make PTransforms // unkeyed -class KeyedPValueTrackingVisitor implements PipelineVisitor { +class KeyedPValueTrackingVisitor extends PipelineVisitor.Defaults { private static final Set> PRODUCES_KEYED_OUTPUTS = ImmutableSet.of( @@ -90,9 +90,6 @@ public void leaveCompositeTransform(TransformHierarchy.Node node) { } } - @Override - public void visitPrimitiveTransform(TransformHierarchy.Node node) {} - @Override public void visitValue(PValue value, TransformHierarchy.Node producer) { boolean inputsAreKeyed = true; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java index 854b67460ae2..50910b5bab5d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java @@ -112,7 +112,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { BatchTransformTranslator typedTranslator = (BatchTransformTranslator) translator; // create the applied PTransform on the batchContext - batchContext.setCurrentTransform(node.toAppliedPTransform()); + batchContext.setCurrentTransform(node.toAppliedPTransform(getPipeline())); typedTranslator.translateNode(typedTransform, batchContext); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java index 80ef7bb32acd..ca12615be03c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java @@ -38,7 +38,6 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.values.PValue; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.client.program.DetachedEnvironment; import org.slf4j.Logger; @@ -199,10 +198,7 @@ private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pip // have just recorded the full names during apply time. if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) { final SortedSet ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>(); - pipeline.traverseTopologically(new Pipeline.PipelineVisitor() { - @Override - public void visitValue(PValue value, TransformHierarchy.Node producer) { - } + pipeline.traverseTopologically(new Pipeline.PipelineVisitor.Defaults() { @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { @@ -218,10 +214,6 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { } return CompositeBehavior.ENTER_TRANSFORM; } - - @Override - public void leaveCompositeTransform(TransformHierarchy.Node node) { - } }); LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} " diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java index 53a1fa160ad1..8da68c5fc11e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java @@ -188,7 +188,7 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { StreamTransformTranslator typedTranslator = (StreamTransformTranslator) translator; // create the applied PTransform on the streamingContext - streamingContext.setCurrentTransform(node.toAppliedPTransform()); + streamingContext.setCurrentTransform(node.toAppliedPTransform(getPipeline())); typedTranslator.translateNode(typedTransform, streamingContext); } @@ -203,7 +203,7 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { @SuppressWarnings("unchecked") StreamTransformTranslator typedTranslator = (StreamTransformTranslator) translator; - streamingContext.setCurrentTransform(node.toAppliedPTransform()); + streamingContext.setCurrentTransform(node.toAppliedPTransform(getPipeline())); return typedTranslator.canTranslate(typedTransform, streamingContext); } 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 index 840bda846ed6..6d7a0f847ca2 100644 --- 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 @@ -431,18 +431,18 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { transform, node.getFullName()); LOG.debug("Translating {}", transform); - currentTransform = node.toAppliedPTransform(); + currentTransform = node.toAppliedPTransform(getPipeline()); translator.translate(transform, this); currentTransform = null; } @Override public void visitValue(PValue value, TransformHierarchy.Node producer) { - producers.put(value, producer.toAppliedPTransform()); + producers.put(value, producer.toAppliedPTransform(getPipeline())); LOG.debug("Checking translation of {}", value); if (!producer.isCompositeNode()) { // Primitive transforms are the only ones assigned step names. - asOutputReference(value, producer.toAppliedPTransform()); + asOutputReference(value, producer.toAppliedPTransform(getPipeline())); } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 2ef87374956d..cce6ce79b883 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -729,7 +729,7 @@ private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pip if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) { final SortedSet ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>(); pipeline.traverseTopologically( - new PipelineVisitor() { + new PipelineVisitor.Defaults() { @Override public void visitValue(PValue value, TransformHierarchy.Node producer) {} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index 8c02f0f0e90d..9e2426ef8381 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -404,7 +404,7 @@ > void doVisitTransform( @SuppressWarnings("unchecked") TransformEvaluator evaluator = translate(node, transform, transformClass); LOG.info("Evaluating {}", transform); - AppliedPTransform appliedTransform = node.toAppliedPTransform(); + AppliedPTransform appliedTransform = node.toAppliedPTransform(getPipeline()); ctxt.setCurrentTransform(appliedTransform); evaluator.evaluate(transform, ctxt); ctxt.setCurrentTransform(null); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/TrackStreamingSourcesTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/TrackStreamingSourcesTest.java index 33a636ad65f8..e8a59510cdb9 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/TrackStreamingSourcesTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/TrackStreamingSourcesTest.java @@ -147,6 +147,12 @@ private void assertSourceIds(List streamingSources) { assertThat(streamingSources, containsInAnyOrder(expected)); } + @Override + public void enterPipeline(Pipeline p) { + super.enterPipeline(p); + evaluator.enterPipeline(p); + } + @Override public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { return evaluator.enterCompositeTransform(node); @@ -156,7 +162,7 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { public void visitPrimitiveTransform(TransformHierarchy.Node node) { PTransform transform = node.getTransform(); if (transform.getClass() == transformClassToAssert) { - AppliedPTransform appliedTransform = node.toAppliedPTransform(); + AppliedPTransform appliedTransform = node.toAppliedPTransform(getPipeline()); ctxt.setCurrentTransform(appliedTransform); //noinspection unchecked Dataset dataset = ctxt.borrowDataset((PTransform) transform); @@ -166,6 +172,12 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { evaluator.visitPrimitiveTransform(node); } } + + @Override + public void leavePipeline(Pipeline p) { + super.leavePipeline(p); + evaluator.leavePipeline(p); + } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java index 83496a5302ad..bdf8a12aa2bf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk; +import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; @@ -205,7 +206,7 @@ private void checkNoMoreMatches(final List overrides) { public CompositeBehavior enterCompositeTransform(Node node) { if (!node.isRootNode()) { for (PTransformOverride override : overrides) { - if (override.getMatcher().matches(node.toAppliedPTransform())) { + if (override.getMatcher().matches(node.toAppliedPTransform(getPipeline()))) { matched.put(node, override); } } @@ -227,7 +228,7 @@ public void leaveCompositeTransform(Node node) { @Override public void visitPrimitiveTransform(Node node) { for (PTransformOverride override : overrides) { - if (override.getMatcher().matches(node.toAppliedPTransform())) { + if (override.getMatcher().matches(node.toAppliedPTransform(getPipeline()))) { matched.put(node, override); } } @@ -238,7 +239,7 @@ public void visitPrimitiveTransform(Node node) { private void replace(final PTransformOverride override) { final Set matches = new HashSet<>(); final Set freedNodes = new HashSet<>(); - transforms.visit( + traverseTopologically( new PipelineVisitor.Defaults() { @Override public CompositeBehavior enterCompositeTransform(Node node) { @@ -247,7 +248,8 @@ public CompositeBehavior enterCompositeTransform(Node node) { freedNodes.add(node); return CompositeBehavior.ENTER_TRANSFORM; } - if (!node.isRootNode() && override.getMatcher().matches(node.toAppliedPTransform())) { + if (!node.isRootNode() + && override.getMatcher().matches(node.toAppliedPTransform(getPipeline()))) { matches.add(node); // This node will be freed. When we visit any of its children, they will also be freed freedNodes.add(node); @@ -259,7 +261,7 @@ public CompositeBehavior enterCompositeTransform(Node node) { public void visitPrimitiveTransform(Node node) { if (freedNodes.contains(node.getEnclosingNode())) { freedNodes.add(node); - } else if (override.getMatcher().matches(node.toAppliedPTransform())) { + } else if (override.getMatcher().matches(node.toAppliedPTransform(getPipeline()))) { matches.add(node); freedNodes.add(node); } @@ -334,8 +336,14 @@ public void setCoderRegistry(CoderRegistry coderRegistry) { @Internal public interface PipelineVisitor { /** - * Called for each composite transform after all topological predecessors have been visited - * but before any of its component transforms. + * Called before visiting anything values or transforms, as many uses of a visitor require + * access to the {@link Pipeline} object itself. + */ + void enterPipeline(Pipeline p); + + /** + * Called for each composite transform after all topological predecessors have been visited but + * before any of its component transforms. * *

The return value controls whether or not child transforms are visited. */ @@ -359,6 +367,11 @@ public interface PipelineVisitor { */ void visitValue(PValue value, TransformHierarchy.Node producer); + /** + * Called when all values and transforms in a {@link Pipeline} have been visited. + */ + void leavePipeline(Pipeline pipeline); + /** * Control enum for indicating whether or not a traversal should process the contents of * a composite transform or not. @@ -373,6 +386,18 @@ enum CompositeBehavior { * User implementations can override just those methods they are interested in. */ class Defaults implements PipelineVisitor { + + private Pipeline pipeline; + + protected Pipeline getPipeline() { + return pipeline; + } + + @Override + public void enterPipeline(Pipeline pipeline) { + this.pipeline = checkNotNull(pipeline); + } + @Override public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { return CompositeBehavior.ENTER_TRANSFORM; @@ -386,6 +411,11 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { } @Override public void visitValue(PValue value, TransformHierarchy.Node producer) { } + + @Override + public void leavePipeline(Pipeline pipeline) { + this.pipeline = null; + } } } @@ -406,7 +436,9 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { } */ @Internal public void traverseTopologically(PipelineVisitor visitor) { + visitor.enterPipeline(this); transforms.visit(visitor); + visitor.leavePipeline(this); } /** @@ -444,7 +476,7 @@ OutputT applyTransform(String name, InputT input, ///////////////////////////////////////////////////////////////////////////// // Below here are internal operations, never called by users. - private final TransformHierarchy transforms = new TransformHierarchy(this); + private final TransformHierarchy transforms = new TransformHierarchy(); private Set usedFullNames = new HashSet<>(); private CoderRegistry coderRegistry; private final List unstableNames = new ArrayList<>(); @@ -495,7 +527,7 @@ void applyReplacement( PTransformOverrideFactory replacementFactory) { PTransformReplacement replacement = replacementFactory.getReplacementTransform( - (AppliedPTransform) original.toAppliedPTransform()); + (AppliedPTransform) original.toAppliedPTransform(this)); if (replacement.getTransform() == original.getTransform()) { return; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index fac558bc8432..2f0e8efd7de8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -56,7 +56,6 @@ public class TransformHierarchy { private static final Logger LOG = LoggerFactory.getLogger(TransformHierarchy.class); - private final Pipeline pipeline; private final Node root; private final Map unexpandedInputs; private final Map producers; @@ -65,8 +64,7 @@ public class TransformHierarchy { // Maintain a stack based on the enclosing nodes private Node current; - public TransformHierarchy(Pipeline pipeline) { - this.pipeline = pipeline; + public TransformHierarchy() { producers = new HashMap<>(); producerInput = new HashMap<>(); unexpandedInputs = new HashMap<>(); @@ -453,7 +451,7 @@ public Map, PValue> getOutputs() { /** * Returns the {@link AppliedPTransform} representing this {@link Node}. */ - public AppliedPTransform toAppliedPTransform() { + public AppliedPTransform toAppliedPTransform(Pipeline pipeline) { return AppliedPTransform.of( getFullName(), inputs, outputs, (PTransform) getTransform(), pipeline); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java index 125e15902d17..1197d1b04eb6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java @@ -79,7 +79,7 @@ public class TransformHierarchyTest implements Serializable { @Before public void setup() { - hierarchy = new TransformHierarchy(pipeline); + hierarchy = new TransformHierarchy(); } @Test From a7ad0c2b06df5fc281eb91f73053396a61b11f53 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 18 May 2017 09:09:33 -0700 Subject: [PATCH 023/159] Fix GroupAlsoByWindowEvaluatorFactory doc --- .../GroupAlsoByWindowEvaluatorFactory.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 84be15dd19cd..78ef7fe5d6da 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -26,8 +26,6 @@ import java.util.Collection; import org.apache.beam.runners.core.GroupAlsoByWindowsAggregators; import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly; -import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; -import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.OutputWindowedValue; import org.apache.beam.runners.core.ReduceFnRunner; @@ -57,7 +55,7 @@ /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the - * {@link GroupByKeyOnly} {@link PTransform}. + * {@link DirectGroupAlsoByWindow} {@link PTransform}. */ class GroupAlsoByWindowEvaluatorFactory implements TransformEvaluatorFactory { private final EvaluationContext evaluationContext; @@ -92,8 +90,9 @@ private TransformEvaluator> createEvaluator( } /** - * A transform evaluator for the pseudo-primitive {@link GroupAlsoByWindow}. Windowing is ignored; - * all input should be in the global window since all output will be as well. + * A transform evaluator for the pseudo-primitive {@link DirectGroupAlsoByWindow}. The window of + * the input {@link KeyedWorkItem} is ignored; it should be in the global window, as element + * windows are reified in the {@link KeyedWorkItem#elementsIterable()}. * * @see GroupByKeyViaGroupByKeyOnly */ @@ -173,7 +172,7 @@ public void processElement(WindowedValue> element) throws Ex stateInternals, timerInternals, new OutputWindowedValueToBundle<>(bundle), - new UnsupportedSideInputReader("GroupAlsoByWindow"), + new UnsupportedSideInputReader(DirectGroupAlsoByWindow.class.getSimpleName()), reduceFn, evaluationContext.getPipelineOptions()); @@ -226,8 +225,9 @@ public boolean apply(WindowedValue input) { // The element is too late for this window. droppedDueToLateness.inc(); WindowTracing.debug( - "GroupAlsoByWindow: Dropping element at {} for key: {}; " + "{}: Dropping element at {} for key: {}; " + "window: {} since it is too far behind inputWatermark: {}", + DirectGroupAlsoByWindow.class.getSimpleName(), input.getTimestamp(), key, window, @@ -264,7 +264,9 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - throw new UnsupportedOperationException("GroupAlsoByWindow should not use tagged outputs"); + throw new UnsupportedOperationException( + String.format( + "%s should not use tagged outputs", DirectGroupAlsoByWindow.class.getSimpleName())); } } } From 143334ca532076c839e5699171c8d9db8743b6a2 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 12:04:53 -0700 Subject: [PATCH 024/159] Minor touch-ups to new BoundedWindow javadoc --- .../transforms/windowing/BoundedWindow.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java index 506753dc4f77..92fa3c59f62c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java @@ -21,19 +21,19 @@ import org.joda.time.Instant; /** - * A {@code BoundedWindow} represents window information assigned to data elements. + * A {@link BoundedWindow} represents window information assigned to data elements. * - *

It has one method {@link #maxTimestamp()} to define an upper bound(included) of element - * timestamp. Elements within the timestamp range are placed in this window. - * Technically a window may not have a lower bound. Upper bound is mandatory as watermark, - * which also controls discarding data and expiring the window, is based on event timestamp. + *

It has one method {@link #maxTimestamp()} to define an upper bound (inclusive) for element + * timestamps. A {@link WindowFn} must assign an element only to windows where {@link + * #maxTimestamp()} is greater than or equal to the element timestamp. When the watermark passes the + * maximum timestamp, all data for a window is estimated to be received. * - *

At some point in time, usually when watermark passes, - * all data with the same window will arrive and can be processed together. + *

A window does not need to have a lower bound. Only the upper bound is mandatory because it + * governs management of triggering and discarding of the window. * - *

Windows must also implement {@link Object#equals} and - * {@link Object#hashCode} such that windows that are logically equal will - * be treated as equal by {@code equals()} and {@code hashCode()}. + *

Windows must also implement {@link Object#equals} and {@link Object#hashCode} such that + * windows that are logically equal will be treated as equal by {@code equals()} and {@code + * hashCode()}. */ public abstract class BoundedWindow { // The min and max timestamps that won't overflow when they are converted to From 10357c20ba62dab873749f135d23c4dda4033cd7 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 19 May 2017 14:06:12 -0700 Subject: [PATCH 025/159] Remove Unused DoFnInfo methods --- .../beam/runners/dataflow/util/DoFnInfo.java | 25 ------------------- .../control/ProcessBundleHandlerTest.java | 2 +- 2 files changed, 1 insertion(+), 26 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java index bd2742f147da..4a26795ec138 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java @@ -53,25 +53,6 @@ public static DoFnInfo forFn( doFn, windowingStrategy, sideInputViews, inputCoder, mainOutput, outputMap); } - /** TODO: remove this when Dataflow worker uses the DoFn overload. */ - @Deprecated - @SuppressWarnings("unchecked") - public static DoFnInfo forFn( - Serializable doFn, - WindowingStrategy windowingStrategy, - Iterable> sideInputViews, - Coder inputCoder, - long mainOutput, - Map> outputMap) { - return forFn( - (DoFn) doFn, - windowingStrategy, - sideInputViews, - inputCoder, - mainOutput, - outputMap); - } - public DoFnInfo withFn(DoFn newFn) { return DoFnInfo.forFn(newFn, windowingStrategy, @@ -96,12 +77,6 @@ private DoFnInfo( this.outputMap = outputMap; } - /** TODO: remove this when Dataflow worker uses {@link #getDoFn}. */ - @Deprecated - public Serializable getFn() { - return doFn; - } - /** Returns the embedded function. */ public DoFn getDoFn() { return doFn; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java index 748ffea1f13c..f40572843bee 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java @@ -340,7 +340,7 @@ public void testCreatingAndProcessingDoFn() throws Exception { new TestDoFn(), WindowingStrategy.globalDefault(), ImmutableList.of(), - STRING_CODER, + StringUtf8Coder.of(), mainOutputId, ImmutableMap.of( mainOutputId, TestDoFn.mainOutput, From 888a5e6ab9c1c83ee06281de4c906be69c076286 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 19 May 2017 14:24:07 -0700 Subject: [PATCH 026/159] Extract the Main Input PCollection in ParDos --- .../beam/runners/core/construction/ParDos.java | 16 ++++++++++++++++ .../runners/core/construction/ParDosTest.java | 4 ++++ 2 files changed, 20 insertions(+) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java index 4752bd1a8940..2ecc04161fed 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java @@ -23,6 +23,8 @@ import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; import com.google.common.base.Optional; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; import com.google.protobuf.Any; import com.google.protobuf.ByteString; import com.google.protobuf.BytesValue; @@ -161,6 +163,20 @@ public static TupleTag getMainOutputTag(ParDoPayload payload) return doFnAndMainOutputTagFromProto(payload.getDoFn()).getMainOutputTag(); } + public static RunnerApi.PCollection getMainInput( + RunnerApi.PTransform ptransform, Components components) throws IOException { + checkArgument( + ptransform.getSpec().getUrn().equals(PAR_DO_PAYLOAD_URN), + "Unexpected payload type %s", + ptransform.getSpec().getUrn()); + ParDoPayload payload = ptransform.getSpec().getParameter().unpack(ParDoPayload.class); + String mainInputId = + Iterables.getOnlyElement( + Sets.difference( + ptransform.getInputsMap().keySet(), payload.getSideInputsMap().keySet())); + return components.getPcollectionsOrThrow(ptransform.getInputsOrThrow(mainInputId)); + } + // TODO: Implement private static StateSpec toProto(StateDeclaration state) { throw new UnsupportedOperationException("Not yet supported"); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java index 74edec15af81..b6f0b7d9d68e 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java @@ -149,6 +149,10 @@ public void toAndFromTransformProto() throws Exception { view.getWindowingStrategyInternal().fixDefaults())); assertThat(restoredView.getCoderInternal(), equalTo(view.getCoderInternal())); } + String mainInputId = components.registerPCollection(mainInput); + assertThat( + ParDos.getMainInput(protoTransform, protoComponents), + equalTo(protoComponents.getPcollectionsOrThrow(mainInputId))); } private static class DropElementsFn extends DoFn, Void> { From 43e291015b07fe8f98cb70131bb2482dfa633c7c Mon Sep 17 00:00:00 2001 From: Stephen Sisk Date: Mon, 22 May 2017 11:36:09 -0700 Subject: [PATCH 027/159] Cleanup k8s scripts naming & don't create insecure svc by default These scripts setup a internet-accessible service by default, which is insecure since we rely on firewalls for securing the data stores. --- .../cassandra/LargeITCluster/{start-up.sh => setup.sh} | 1 - .test-infra/kubernetes/cassandra/LargeITCluster/teardown.sh | 1 - .../cassandra/SmallITCluster/{start-up.sh => setup.sh} | 1 - .test-infra/kubernetes/cassandra/SmallITCluster/teardown.sh | 1 - .../LargeProductionCluster/{start-up.sh => setup.sh} | 1 - .../kubernetes/elasticsearch/LargeProductionCluster/teardown.sh | 1 - .../elasticsearch/SmallITCluster/{start-up.sh => setup.sh} | 1 - .test-infra/kubernetes/elasticsearch/SmallITCluster/teardown.sh | 1 - 8 files changed, 8 deletions(-) rename .test-infra/kubernetes/cassandra/LargeITCluster/{start-up.sh => setup.sh} (94%) rename .test-infra/kubernetes/cassandra/SmallITCluster/{start-up.sh => setup.sh} (94%) rename .test-infra/kubernetes/elasticsearch/LargeProductionCluster/{start-up.sh => setup.sh} (94%) rename .test-infra/kubernetes/elasticsearch/SmallITCluster/{start-up.sh => setup.sh} (94%) diff --git a/.test-infra/kubernetes/cassandra/LargeITCluster/start-up.sh b/.test-infra/kubernetes/cassandra/LargeITCluster/setup.sh similarity index 94% rename from .test-infra/kubernetes/cassandra/LargeITCluster/start-up.sh rename to .test-infra/kubernetes/cassandra/LargeITCluster/setup.sh index 7341209ccaf8..7bc08099a9ad 100644 --- a/.test-infra/kubernetes/cassandra/LargeITCluster/start-up.sh +++ b/.test-infra/kubernetes/cassandra/LargeITCluster/setup.sh @@ -18,5 +18,4 @@ set -e # Create Cassandra services and statefulset. -kubectl create -f cassandra-service-for-local-dev.yaml kubectl create -f cassandra-svc-statefulset.yaml diff --git a/.test-infra/kubernetes/cassandra/LargeITCluster/teardown.sh b/.test-infra/kubernetes/cassandra/LargeITCluster/teardown.sh index 367b6049fcbe..3d040a637a0c 100644 --- a/.test-infra/kubernetes/cassandra/LargeITCluster/teardown.sh +++ b/.test-infra/kubernetes/cassandra/LargeITCluster/teardown.sh @@ -20,6 +20,5 @@ set -e # Delete Cassandra services and statefulset. kubectl delete -f cassandra-svc-statefulset.yaml -kubectl delete -f cassandra-service-for-local-dev.yaml # Delete the persistent storage media for the PersistentVolumes kubectl delete pvc -l app=cassandra diff --git a/.test-infra/kubernetes/cassandra/SmallITCluster/start-up.sh b/.test-infra/kubernetes/cassandra/SmallITCluster/setup.sh similarity index 94% rename from .test-infra/kubernetes/cassandra/SmallITCluster/start-up.sh rename to .test-infra/kubernetes/cassandra/SmallITCluster/setup.sh index 9377a9c190a0..fad6df05ca24 100644 --- a/.test-infra/kubernetes/cassandra/SmallITCluster/start-up.sh +++ b/.test-infra/kubernetes/cassandra/SmallITCluster/setup.sh @@ -18,6 +18,5 @@ set -e # Create Cassandra services and Replication controller. -kubectl create -f cassandra-service-for-local-dev.yaml kubectl create -f cassandra-svc-rc.yaml diff --git a/.test-infra/kubernetes/cassandra/SmallITCluster/teardown.sh b/.test-infra/kubernetes/cassandra/SmallITCluster/teardown.sh index f4ad0be90456..f538a753eb0a 100644 --- a/.test-infra/kubernetes/cassandra/SmallITCluster/teardown.sh +++ b/.test-infra/kubernetes/cassandra/SmallITCluster/teardown.sh @@ -19,4 +19,3 @@ set -e # Delete Cassandra services and Replication controller. kubectl delete -f cassandra-svc-rc.yaml -kubectl delete -f cassandra-service-for-local-dev.yaml diff --git a/.test-infra/kubernetes/elasticsearch/LargeProductionCluster/start-up.sh b/.test-infra/kubernetes/elasticsearch/LargeProductionCluster/setup.sh similarity index 94% rename from .test-infra/kubernetes/elasticsearch/LargeProductionCluster/start-up.sh rename to .test-infra/kubernetes/elasticsearch/LargeProductionCluster/setup.sh index 93022c724400..9fbb6c31ad57 100644 --- a/.test-infra/kubernetes/elasticsearch/LargeProductionCluster/start-up.sh +++ b/.test-infra/kubernetes/elasticsearch/LargeProductionCluster/setup.sh @@ -18,5 +18,4 @@ set -e # Create Elasticsearch services and deployments. -kubectl create -f elasticsearch-service-for-local-dev.yaml kubectl create -f es-services-deployments.yaml diff --git a/.test-infra/kubernetes/elasticsearch/LargeProductionCluster/teardown.sh b/.test-infra/kubernetes/elasticsearch/LargeProductionCluster/teardown.sh index bdc9ab9341a6..18568a3e4230 100644 --- a/.test-infra/kubernetes/elasticsearch/LargeProductionCluster/teardown.sh +++ b/.test-infra/kubernetes/elasticsearch/LargeProductionCluster/teardown.sh @@ -18,4 +18,3 @@ set -e # Delete elasticsearch services and deployments. kubectl delete -f es-services-deployments.yaml -kubectl delete -f elasticsearch-service-for-local-dev.yaml diff --git a/.test-infra/kubernetes/elasticsearch/SmallITCluster/start-up.sh b/.test-infra/kubernetes/elasticsearch/SmallITCluster/setup.sh similarity index 94% rename from .test-infra/kubernetes/elasticsearch/SmallITCluster/start-up.sh rename to .test-infra/kubernetes/elasticsearch/SmallITCluster/setup.sh index 2d6522ea9c54..e8cf275d9729 100644 --- a/.test-infra/kubernetes/elasticsearch/SmallITCluster/start-up.sh +++ b/.test-infra/kubernetes/elasticsearch/SmallITCluster/setup.sh @@ -18,6 +18,5 @@ set -e # Create Elasticsearch services and deployments. -kubectl create -f elasticsearch-service-for-local-dev.yaml kubectl create -f elasticsearch-svc-rc.yaml diff --git a/.test-infra/kubernetes/elasticsearch/SmallITCluster/teardown.sh b/.test-infra/kubernetes/elasticsearch/SmallITCluster/teardown.sh index 61c079fed894..079141d8cd9b 100644 --- a/.test-infra/kubernetes/elasticsearch/SmallITCluster/teardown.sh +++ b/.test-infra/kubernetes/elasticsearch/SmallITCluster/teardown.sh @@ -18,4 +18,3 @@ set -e # Delete elasticsearch services and deployments. kubectl delete -f elasticsearch-svc-rc.yaml -kubectl delete -f elasticsearch-service-for-local-dev.yaml From 13d20122bc0b20e1348714409fbc3b6b1f1c1e4f Mon Sep 17 00:00:00 2001 From: Sourabh Bajaj Date: Mon, 22 May 2017 13:00:52 -0700 Subject: [PATCH 028/159] [BEAM-1999] Remove resolved todo from value providers --- sdks/python/apache_beam/options/value_provider.py | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/python/apache_beam/options/value_provider.py b/sdks/python/apache_beam/options/value_provider.py index 40bddba5a19c..f7630bd8d326 100644 --- a/sdks/python/apache_beam/options/value_provider.py +++ b/sdks/python/apache_beam/options/value_provider.py @@ -82,7 +82,6 @@ def get(self): value = self.default_value return value - # TODO(BEAM-1999): Remove _unused_options_id @classmethod def set_runtime_options(cls, pipeline_options): RuntimeValueProvider.runtime_options = pipeline_options From 39e8d37ee95709d07a74c9df866059871d5585f5 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Mon, 22 May 2017 13:03:38 -0700 Subject: [PATCH 029/159] Update Dataflow BATCH_ENVIRONMENT_MAJOR_VERSION to 6 --- sdks/python/apache_beam/runners/dataflow/dataflow_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 0ecd22ac80b2..a05e582b05f4 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -62,7 +62,7 @@ class DataflowRunner(PipelineRunner): # Environment version information. It is passed to the service during a # a job submission and is used by the service to establish what features # are expected by the workers. - BATCH_ENVIRONMENT_MAJOR_VERSION = '5' + BATCH_ENVIRONMENT_MAJOR_VERSION = '6' STREAMING_ENVIRONMENT_MAJOR_VERSION = '0' def __init__(self, cache=None): From 7f35c98b5f77069bd21dc7dea4a7d046883d13a6 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 18 May 2017 10:23:35 -0700 Subject: [PATCH 030/159] Add ReadTranslator This translates Read transforms to ReadPayloads and back --- .../core/construction/ReadTranslator.java | 127 +++++++++++++ .../core/construction/ReadTranslatorTest.java | 179 ++++++++++++++++++ .../beam/runners/dataflow/ReadTranslator.java | 5 +- .../apache/beam/sdk/io/CountingSource.java | 42 ++++ 4 files changed, 350 insertions(+), 3 deletions(-) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslator.java create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslatorTest.java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslator.java new file mode 100644 index 000000000000..f94493862b2e --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslator.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.core.construction; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.IsBounded; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.ReadPayload; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.Source; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.SerializableUtils; + +/** + * Methods for translating {@link Read.Bounded} and {@link Read.Unbounded} + * {@link PTransform PTransforms} into {@link ReadPayload} protos. + */ +public class ReadTranslator { + private static final String JAVA_SERIALIZED_BOUNDED_SOURCE = "urn:beam:java:boundedsource:v1"; + private static final String JAVA_SERIALIZED_UNBOUNDED_SOURCE = "urn:beam:java:unboundedsource:v1"; + + public static ReadPayload toProto(Read.Bounded read) { + return ReadPayload.newBuilder() + .setIsBounded(IsBounded.BOUNDED) + .setSource(toProto(read.getSource())) + .build(); + } + + public static ReadPayload toProto(Read.Unbounded read) { + return ReadPayload.newBuilder() + .setIsBounded(IsBounded.UNBOUNDED) + .setSource(toProto(read.getSource())) + .build(); + } + + public static SdkFunctionSpec toProto(Source source) { + if (source instanceof BoundedSource) { + return toProto((BoundedSource) source); + } else if (source instanceof UnboundedSource) { + return toProto((UnboundedSource) source); + } else { + throw new IllegalArgumentException( + String.format("Unknown %s type %s", Source.class.getSimpleName(), source.getClass())); + } + } + + private static SdkFunctionSpec toProto(BoundedSource source) { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(JAVA_SERIALIZED_BOUNDED_SOURCE) + .setParameter( + Any.pack( + BytesValue.newBuilder() + .setValue( + ByteString.copyFrom(SerializableUtils.serializeToByteArray(source))) + .build()))) + .build(); + } + + public static BoundedSource boundedSourceFromProto(ReadPayload payload) + throws InvalidProtocolBufferException { + checkArgument(payload.getIsBounded().equals(IsBounded.BOUNDED)); + return (BoundedSource) SerializableUtils.deserializeFromByteArray( + payload + .getSource() + .getSpec() + .getParameter() + .unpack(BytesValue.class) + .getValue() + .toByteArray(), + "BoundedSource"); + } + + private static SdkFunctionSpec toProto(UnboundedSource source) { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(JAVA_SERIALIZED_UNBOUNDED_SOURCE) + .setParameter( + Any.pack( + BytesValue.newBuilder() + .setValue( + ByteString.copyFrom(SerializableUtils.serializeToByteArray(source))) + .build()))) + .build(); + } + + public static UnboundedSource unboundedSourceFromProto(ReadPayload payload) + throws InvalidProtocolBufferException { + checkArgument(payload.getIsBounded().equals(IsBounded.UNBOUNDED)); + return (UnboundedSource) SerializableUtils.deserializeFromByteArray( + payload + .getSource() + .getSpec() + .getParameter() + .unpack(BytesValue.class) + .getValue() + .toByteArray(), + "BoundedSource"); + } + +} diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslatorTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslatorTest.java new file mode 100644 index 000000000000..a603e342bc9f --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslatorTest.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertThat; +import static org.junit.Assume.assumeThat; + +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.ReadPayload; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.CountingSource; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.Source; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; +import org.apache.beam.sdk.options.PipelineOptions; +import org.hamcrest.Matchers; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; + +/** + * Tests for {@link ReadTranslator}. + */ +@RunWith(Parameterized.class) +public class ReadTranslatorTest { + + @Parameters(name = "{index}: {0}") + public static Iterable> data() { + return ImmutableList.>of( + CountingSource.unbounded(), + CountingSource.upTo(100L), + new TestBoundedSource(), + new TestUnboundedSource()); + } + + @Parameter(0) + public Source source; + + @Test + public void testToFromProtoBounded() throws Exception { + // TODO: Split into two tests. + assumeThat(source, instanceOf(BoundedSource.class)); + BoundedSource boundedSource = (BoundedSource) this.source; + Read.Bounded boundedRead = Read.from(boundedSource); + ReadPayload payload = ReadTranslator.toProto(boundedRead); + assertThat(payload.getIsBounded(), equalTo(RunnerApi.IsBounded.BOUNDED)); + BoundedSource deserializedSource = ReadTranslator.boundedSourceFromProto(payload); + assertThat(deserializedSource, Matchers.>equalTo(source)); + } + + @Test + public void testToFromProtoUnbounded() throws Exception { + assumeThat(source, instanceOf(UnboundedSource.class)); + UnboundedSource unboundedSource = (UnboundedSource) this.source; + Read.Unbounded unboundedRead = Read.from(unboundedSource); + ReadPayload payload = ReadTranslator.toProto(unboundedRead); + assertThat(payload.getIsBounded(), equalTo(RunnerApi.IsBounded.UNBOUNDED)); + UnboundedSource deserializedSource = ReadTranslator.unboundedSourceFromProto(payload); + assertThat(deserializedSource, Matchers.>equalTo(source)); + } + + private static class TestBoundedSource extends BoundedSource { + @Override + public List> split( + long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public BoundedReader createReader(PipelineOptions options) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void validate() {} + + @Override + public Coder getDefaultOutputCoder() { + return StringUtf8Coder.of(); + } + + @Override + public boolean equals(Object other) { + return other != null && other.getClass().equals(TestBoundedSource.class); + } + + @Override + public int hashCode() { + return TestBoundedSource.class.hashCode(); + } + } + + private static class TestUnboundedSource extends UnboundedSource { + @Override + public void validate() {} + + @Override + public Coder getDefaultOutputCoder() { + return ByteArrayCoder.of(); + } + + @Override + public List> split( + int desiredNumSplits, PipelineOptions options) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public UnboundedReader createReader( + PipelineOptions options, @Nullable CheckpointMark checkpointMark) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public Coder getCheckpointMarkCoder() { + return new TestCheckpointMarkCoder(); + } + + @Override + public boolean equals(Object other) { + return other != null && other.getClass().equals(TestUnboundedSource.class); + } + + @Override + public int hashCode() { + return TestUnboundedSource.class.hashCode(); + } + + private class TestCheckpointMarkCoder extends AtomicCoder { + @Override + public void encode(CheckpointMark value, OutputStream outStream) + throws CoderException, IOException { + throw new UnsupportedOperationException(); + } + + @Override + public CheckpointMark decode(InputStream inStream) throws CoderException, IOException { + throw new UnsupportedOperationException(); + } + } + } +} diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReadTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReadTranslator.java index 30ecbf581d2a..0b22d7e8ceed 100755 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReadTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReadTranslator.java @@ -40,9 +40,8 @@ public void translate(Read.Bounded transform, TranslationContext context) { translateReadHelper(transform.getSource(), transform, context); } - public static void translateReadHelper(Source source, - PTransform transform, - TranslationContext context) { + public static void translateReadHelper( + Source source, PTransform transform, TranslationContext context) { try { StepTranslationContext stepContext = context.addStep(transform, "ParallelRead"); stepContext.addInput(PropertyNames.FORMAT, PropertyNames.CUSTOM_SOURCE_FORMAT); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java index 81082e547f77..6202c2b5fad5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.List; import java.util.NoSuchElementException; +import java.util.Objects; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DefaultCoder; @@ -136,6 +137,16 @@ static class NowTimestampFn implements SerializableFunction { public Instant apply(Long input) { return Instant.now(); } + + @Override + public boolean equals(Object other) { + return other instanceof NowTimestampFn; + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } } /** @@ -180,6 +191,21 @@ public org.apache.beam.sdk.io.BoundedSource.BoundedReader createReader( public Coder getDefaultOutputCoder() { return VarLongCoder.of(); } + + @Override + public boolean equals(Object other) { + if (!(other instanceof BoundedCountingSource)) { + return false; + } + BoundedCountingSource that = (BoundedCountingSource) other; + return this.getStartOffset() == that.getStartOffset() + && this.getEndOffset() == that.getEndOffset(); + } + + @Override + public int hashCode() { + return Objects.hash(this.getStartOffset(), (int) this.getEndOffset()); + } } /** @@ -341,6 +367,22 @@ public void validate() {} public Coder getDefaultOutputCoder() { return VarLongCoder.of(); } + + public boolean equals(Object other) { + if (!(other instanceof UnboundedCountingSource)) { + return false; + } + UnboundedCountingSource that = (UnboundedCountingSource) other; + return this.start == that.start + && this.stride == that.stride + && this.elementsPerPeriod == that.elementsPerPeriod + && Objects.equals(this.period, that.period) + && Objects.equals(this.timestampFn, that.timestampFn); + } + + public int hashCode() { + return Objects.hash(start, stride, elementsPerPeriod, period, timestampFn); + } } /** From 9336230d2a5c18bae89908bcd60db8ea96b7906d Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 4 Apr 2017 17:43:48 -0700 Subject: [PATCH 031/159] Include Additional PTransform inputs in Transform Nodes Add the value of PTransform.getAdditionalInputs in the inputs of a TransformHierarchy node. Fork the Node constructor to reduce nullability This slightly simplifies the constructor implementation(s). Update the DirectRunner to track main inputs instead of all inputs. --- .../apex/translation/TranslationContext.java | 4 +- .../core/construction/TransformInputs.java | 50 ++++++ .../construction/TransformInputsTest.java | 166 ++++++++++++++++++ .../runners/direct/DirectGraphVisitor.java | 15 +- .../runners/direct/ParDoEvaluatorFactory.java | 9 +- ...ttableProcessElementsEvaluatorFactory.java | 2 + .../direct/StatefulParDoEvaluatorFactory.java | 1 + .../beam/runners/direct/WatermarkManager.java | 17 +- .../runners/direct/ParDoEvaluatorTest.java | 6 +- .../flink/FlinkBatchTranslationContext.java | 3 +- .../FlinkStreamingTranslationContext.java | 3 +- .../spark/translation/EvaluationContext.java | 4 +- .../beam/sdk/runners/TransformHierarchy.java | 28 ++- 13 files changed, 280 insertions(+), 28 deletions(-) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java index aff3863624c4..94d13e177dec 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java @@ -34,6 +34,7 @@ import org.apache.beam.runners.apex.translation.utils.ApexStateInternals.ApexStateBackend; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translation.utils.CoderAdapterStreamCodec; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -93,7 +94,8 @@ public Map, PValue> getInputs() { } public InputT getInput() { - return (InputT) Iterables.getOnlyElement(getCurrentTransform().getInputs().values()); + return (InputT) + Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(getCurrentTransform())); } public Map, PValue> getOutputs() { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java new file mode 100644 index 000000000000..2baf93a3c128 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.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.core.construction; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.common.collect.ImmutableList; +import java.util.Collection; +import java.util.Map; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; + +/** Utilities for extracting subsets of inputs from an {@link AppliedPTransform}. */ +public class TransformInputs { + /** + * Gets all inputs of the {@link AppliedPTransform} that are not returned by {@link + * PTransform#getAdditionalInputs()}. + */ + public static Collection nonAdditionalInputs(AppliedPTransform application) { + ImmutableList.Builder mainInputs = ImmutableList.builder(); + PTransform transform = application.getTransform(); + for (Map.Entry, PValue> input : application.getInputs().entrySet()) { + if (!transform.getAdditionalInputs().containsKey(input.getKey())) { + mainInputs.add(input.getValue()); + } + } + checkArgument( + !mainInputs.build().isEmpty() || application.getInputs().isEmpty(), + "Expected at least one main input if any inputs exist"); + return mainInputs.build(); + } +} diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java new file mode 100644 index 000000000000..f5b2c11e7923 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import static org.junit.Assert.assertThat; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.PTransform; +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.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.hamcrest.Matchers; +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 TransformInputs}. */ +@RunWith(JUnit4.class) +public class TransformInputsTest { + @Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void nonAdditionalInputsWithNoInputSucceeds() { + AppliedPTransform transform = + AppliedPTransform.of( + "input-free", + Collections., PValue>emptyMap(), + Collections., PValue>emptyMap(), + new TestTransform(), + pipeline); + + assertThat(TransformInputs.nonAdditionalInputs(transform), Matchers.empty()); + } + + @Test + public void nonAdditionalInputsWithOneMainInputSucceeds() { + PCollection input = pipeline.apply(GenerateSequence.from(1L)); + AppliedPTransform transform = + AppliedPTransform.of( + "input-single", + Collections., PValue>singletonMap(new TupleTag() {}, input), + Collections., PValue>emptyMap(), + new TestTransform(), + pipeline); + + assertThat( + TransformInputs.nonAdditionalInputs(transform), Matchers.containsInAnyOrder(input)); + } + + @Test + public void nonAdditionalInputsWithMultipleNonAdditionalInputsSucceeds() { + Map, PValue> allInputs = new HashMap<>(); + PCollection mainInts = pipeline.apply("MainInput", Create.of(12, 3)); + allInputs.put(new TupleTag() {}, mainInts); + PCollection voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of())); + allInputs.put(new TupleTag() {}, voids); + AppliedPTransform transform = + AppliedPTransform.of( + "additional-free", + allInputs, + Collections., PValue>emptyMap(), + new TestTransform(), + pipeline); + + assertThat( + TransformInputs.nonAdditionalInputs(transform), + Matchers.containsInAnyOrder(voids, mainInts)); + } + + @Test + public void nonAdditionalInputsWithAdditionalInputsSucceeds() { + Map, PValue> additionalInputs = new HashMap<>(); + additionalInputs.put(new TupleTag() {}, pipeline.apply(Create.of("1, 2", "3"))); + additionalInputs.put(new TupleTag() {}, pipeline.apply(GenerateSequence.from(3L))); + + Map, PValue> allInputs = new HashMap<>(); + PCollection mainInts = pipeline.apply("MainInput", Create.of(12, 3)); + allInputs.put(new TupleTag() {}, mainInts); + PCollection voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of())); + allInputs.put( + new TupleTag() {}, voids); + allInputs.putAll(additionalInputs); + + AppliedPTransform transform = + AppliedPTransform.of( + "additional", + allInputs, + Collections., PValue>emptyMap(), + new TestTransform(additionalInputs), + pipeline); + + assertThat( + TransformInputs.nonAdditionalInputs(transform), + Matchers.containsInAnyOrder(mainInts, voids)); + } + + @Test + public void nonAdditionalInputsWithOnlyAdditionalInputsThrows() { + Map, PValue> additionalInputs = new HashMap<>(); + additionalInputs.put(new TupleTag() {}, pipeline.apply(Create.of("1, 2", "3"))); + additionalInputs.put(new TupleTag() {}, pipeline.apply(GenerateSequence.from(3L))); + + AppliedPTransform transform = + AppliedPTransform.of( + "additional-only", + additionalInputs, + Collections., PValue>emptyMap(), + new TestTransform(additionalInputs), + pipeline); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("at least one"); + TransformInputs.nonAdditionalInputs(transform); + } + + private static class TestTransform extends PTransform { + private final Map, PValue> additionalInputs; + + private TestTransform() { + this(Collections., PValue>emptyMap()); + } + + private TestTransform(Map, PValue> additionalInputs) { + this.additionalInputs = additionalInputs; + } + + @Override + public POutput expand(PInput input) { + return PDone.in(input.getPipeline()); + } + + @Override + public Map, PValue> getAdditionalInputs() { + return additionalInputs; + } + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index 01204e3049dd..ed4282bbdb8b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -21,10 +21,12 @@ import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ListMultimap; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -34,6 +36,8 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the @@ -41,6 +45,7 @@ * input after the upstream transform has produced and committed output. */ class DirectGraphVisitor extends PipelineVisitor.Defaults { + private static final Logger LOG = LoggerFactory.getLogger(DirectGraphVisitor.class); private Map> producers = new HashMap<>(); @@ -83,7 +88,15 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (node.getInputs().isEmpty()) { rootTransforms.add(appliedTransform); } else { - for (PValue value : node.getInputs().values()) { + Collection mainInputs = + TransformInputs.nonAdditionalInputs(node.toAppliedPTransform(getPipeline())); + if (!mainInputs.containsAll(node.getInputs().values())) { + LOG.debug( + "Inputs reduced to {} from {} by removing additional inputs", + mainInputs, + node.getInputs().values()); + } + for (PValue value : mainInputs) { primitiveConsumers.put(value, appliedTransform); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index 74470bfb8b8d..c52091e27cfe 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -20,7 +20,6 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; -import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -79,6 +78,7 @@ public TransformEvaluator forApplication( (TransformEvaluator) createEvaluator( (AppliedPTransform) application, + (PCollection) inputBundle.getPCollection(), inputBundle.getKey(), doFn, transform.getSideInputs(), @@ -102,6 +102,7 @@ public void cleanup() throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( AppliedPTransform, PCollectionTuple, ?> application, + PCollection mainInput, StructuralKey inputBundleKey, DoFn doFn, List> sideInputs, @@ -120,6 +121,7 @@ DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( createParDoEvaluator( application, inputBundleKey, + mainInput, sideInputs, mainOutputTag, additionalOutputTags, @@ -132,6 +134,7 @@ DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( ParDoEvaluator createParDoEvaluator( AppliedPTransform, PCollectionTuple, ?> application, StructuralKey key, + PCollection mainInput, List> sideInputs, TupleTag mainOutputTag, List> additionalOutputTags, @@ -144,8 +147,7 @@ ParDoEvaluator createParDoEvaluator( evaluationContext, stepContext, application, - ((PCollection) Iterables.getOnlyElement(application.getInputs().values())) - .getWindowingStrategy(), + mainInput.getWindowingStrategy(), fn, key, sideInputs, @@ -173,5 +175,4 @@ static Map, PCollection> pcollections(Map, PValue> ou } return pcs; } - } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index dc85d87bc93f..4e7f4db65478 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -116,6 +116,8 @@ public void cleanup() throws Exception { delegateFactory.createParDoEvaluator( application, inputBundle.getKey(), + (PCollection>>) + inputBundle.getPCollection(), transform.getSideInputs(), transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 985c3be4e9e9..e22edd187c53 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -117,6 +117,7 @@ private TransformEvaluator>> createEvaluator( DoFnLifecycleManagerRemovingTransformEvaluator> delegateEvaluator = delegateFactory.createEvaluator( (AppliedPTransform) application, + (PCollection) inputBundle.getPCollection(), inputBundle.getKey(), doFn, application.getTransform().getUnderlyingParDo().getSideInputs(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index 4f1b8319dc2d..b15b52e314de 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -823,10 +823,11 @@ private Collection getInputProcessingWatermarks(AppliedPTransform getInputWatermarks(AppliedPTransform transform) inputWatermarksBuilder.add(THE_END_OF_TIME); } for (PValue pvalue : inputs.values()) { - Watermark producerOutputWatermark = - getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; - inputWatermarksBuilder.add(producerOutputWatermark); + if (graph.getPrimitiveConsumers(pvalue).contains(transform)) { + Watermark producerOutputWatermark = + getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; + inputWatermarksBuilder.add(producerOutputWatermark); + } } List inputCollectionWatermarks = inputWatermarksBuilder.build(); return inputCollectionWatermarks; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index 286e44d1be04..3b2a22ee26f9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -98,7 +98,7 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { when(evaluationContext.createBundle(output)).thenReturn(outputBundle); ParDoEvaluator evaluator = - createEvaluator(singletonView, fn, output); + createEvaluator(singletonView, fn, inputPc, output); IntervalWindow nonGlobalWindow = new IntervalWindow(new Instant(0), new Instant(10_000L)); WindowedValue first = WindowedValue.valueInGlobalWindow(3); @@ -132,6 +132,7 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { private ParDoEvaluator createEvaluator( PCollectionView singletonView, RecorderFn fn, + PCollection input, PCollection output) { when( evaluationContext.createSideInputReader( @@ -156,8 +157,7 @@ private ParDoEvaluator createEvaluator( evaluationContext, stepContext, transform, - ((PCollection) Iterables.getOnlyElement(transform.getInputs().values())) - .getWindowingStrategy(), + input.getWindowingStrategy(), fn, null /* key */, ImmutableList.>of(singletonView), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java index 0439119dfc40..6e7019848b19 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java @@ -20,6 +20,7 @@ import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.Map; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -143,7 +144,7 @@ Map, PValue> getInputs(PTransform transform) { @SuppressWarnings("unchecked") T getInput(PTransform transform) { - return (T) Iterables.getOnlyElement(currentTransform.getInputs().values()); + return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform)); } Map, PValue> getOutputs(PTransform transform) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java index ea5f6b3162af..74a5fb971144 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java @@ -22,6 +22,7 @@ import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.Map; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -113,7 +114,7 @@ public TypeInformation> getTypeInfo(PCollection collecti @SuppressWarnings("unchecked") public T getInput(PTransform transform) { - return (T) Iterables.getOnlyElement(currentTransform.getInputs().values()); + return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform)); } public Map, PValue> getInputs(PTransform transform) { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index 8102926f6daa..0c6c4d1cb660 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -26,6 +26,7 @@ import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.sdk.Pipeline; @@ -103,7 +104,8 @@ public void setCurrentTransform(AppliedPTransform transform) { public T getInput(PTransform transform) { @SuppressWarnings("unchecked") - T input = (T) Iterables.getOnlyElement(getInputs(transform).values()); + T input = + (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(getCurrentTransform())); return input; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index 2f0e8efd7de8..9d73b4576788 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -32,7 +32,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; @@ -68,7 +67,7 @@ public TransformHierarchy() { producers = new HashMap<>(); producerInput = new HashMap<>(); unexpandedInputs = new HashMap<>(); - root = new Node(null, null, "", null); + root = new Node(); current = root; } @@ -252,26 +251,37 @@ public class Node { @VisibleForTesting boolean finishedSpecifying = false; + /** + * Creates the root-level node. The root level node has a null enclosing node, a null transform, + * an empty map of inputs, and a name equal to the empty string. + */ + private Node() { + this.enclosingNode = null; + this.transform = null; + this.fullName = ""; + this.inputs = Collections.emptyMap(); + } + /** * Creates a new Node with the given parent and transform. * - *

EnclosingNode and transform may both be null for a root-level node, which holds all other - * nodes. - * * @param enclosingNode the composite node containing this node * @param transform the PTransform tracked by this node * @param fullName the fully qualified name of the transform * @param input the unexpanded input to the transform */ private Node( - @Nullable Node enclosingNode, - @Nullable PTransform transform, + Node enclosingNode, + PTransform transform, String fullName, - @Nullable PInput input) { + PInput input) { this.enclosingNode = enclosingNode; this.transform = transform; this.fullName = fullName; - this.inputs = input == null ? Collections., PValue>emptyMap() : input.expand(); + ImmutableMap.Builder, PValue> inputs = ImmutableMap.builder(); + inputs.putAll(input.expand()); + inputs.putAll(transform.getAdditionalInputs()); + this.inputs = inputs.build(); } /** From bb3db2acc07a61844a551ea9f793a0fcf4001878 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Wed, 17 May 2017 15:08:50 -0700 Subject: [PATCH 032/159] Add WindowIntoTranslator This translates Window.Assign into a WindowIntoPayload. --- .../construction/WindowIntoTranslator.java | 61 +++++++++ .../construction/WindowingStrategies.java | 33 ++--- .../WindowIntoTranslatorTest.java | 126 ++++++++++++++++++ 3 files changed, 205 insertions(+), 15 deletions(-) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java new file mode 100644 index 000000000000..ea4c9965790c --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import com.google.protobuf.Any; +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.runners.core.construction.PTransforms.TransformPayloadTranslator; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.WindowIntoPayload; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; + +/** + * Utility methods for translating a {@link Window.Assign} to and from {@link RunnerApi} + * representations. + */ +public class WindowIntoTranslator { + + static class WindowAssignTranslator implements TransformPayloadTranslator> { + @Override + public FunctionSpec translate( + AppliedPTransform> transform, SdkComponents components) { + return FunctionSpec.newBuilder() + .setUrn("urn:beam:transform:window:v1") + .setParameter( + Any.pack(WindowIntoTranslator.toProto(transform.getTransform(), components))) + .build(); + } + } + + public static WindowIntoPayload toProto(Window.Assign transform, SdkComponents components) { + return WindowIntoPayload.newBuilder() + .setWindowFn(WindowingStrategies.toProto(transform.getWindowFn(), components)) + .build(); + } + + public static WindowFn getWindowFn(WindowIntoPayload payload) + throws InvalidProtocolBufferException { + SdkFunctionSpec spec = payload.getWindowFn(); + return WindowingStrategies.windowFnFromProto(spec); + } +} diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategies.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategies.java index 395702f51a3e..8dceebbe6cc3 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategies.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategies.java @@ -163,8 +163,7 @@ public static TimestampCombiner timestampCombinerFromProto(RunnerApi.OutputTime * input {@link WindowFn}. */ public static SdkFunctionSpec toProto( - WindowFn windowFn, @SuppressWarnings("unused") SdkComponents components) - throws IOException { + WindowFn windowFn, @SuppressWarnings("unused") SdkComponents components) { return SdkFunctionSpec.newBuilder() // TODO: Set environment ID .setSpec( @@ -245,7 +244,23 @@ public static RunnerApi.WindowingStrategy toProto( throws InvalidProtocolBufferException { SdkFunctionSpec windowFnSpec = proto.getWindowFn(); + WindowFn windowFn = windowFnFromProto(windowFnSpec); + TimestampCombiner timestampCombiner = timestampCombinerFromProto(proto.getOutputTime()); + AccumulationMode accumulationMode = fromProto(proto.getAccumulationMode()); + Trigger trigger = Triggers.fromProto(proto.getTrigger()); + ClosingBehavior closingBehavior = fromProto(proto.getClosingBehavior()); + Duration allowedLateness = Duration.millis(proto.getAllowedLateness()); + + return WindowingStrategy.of(windowFn) + .withAllowedLateness(allowedLateness) + .withMode(accumulationMode) + .withTrigger(trigger) + .withTimestampCombiner(timestampCombiner) + .withClosingBehavior(closingBehavior); + } + public static WindowFn windowFnFromProto(SdkFunctionSpec windowFnSpec) + throws InvalidProtocolBufferException { checkArgument( windowFnSpec.getSpec().getUrn().equals(CUSTOM_WINDOWFN_URN), "Only Java-serialized %s instances are supported, with URN %s. But found URN %s", @@ -258,18 +273,6 @@ public static RunnerApi.WindowingStrategy toProto( windowFnSpec.getSpec().getParameter().unpack(BytesValue.class).getValue().toByteArray(), "WindowFn"); - WindowFn windowFn = (WindowFn) deserializedWindowFn; - TimestampCombiner timestampCombiner = timestampCombinerFromProto(proto.getOutputTime()); - AccumulationMode accumulationMode = fromProto(proto.getAccumulationMode()); - Trigger trigger = Triggers.fromProto(proto.getTrigger()); - ClosingBehavior closingBehavior = fromProto(proto.getClosingBehavior()); - Duration allowedLateness = Duration.millis(proto.getAllowedLateness()); - - return WindowingStrategy.of(windowFn) - .withAllowedLateness(allowedLateness) - .withMode(accumulationMode) - .withTrigger(trigger) - .withTimestampCombiner(timestampCombiner) - .withClosingBehavior(closingBehavior); + return (WindowFn) deserializedWindowFn; } } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.java new file mode 100644 index 000000000000..fbac56516cc1 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.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.core.construction; + +import static com.google.common.base.Preconditions.checkState; +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.ImmutableList; +import com.google.protobuf.InvalidProtocolBufferException; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.beam.sdk.Pipeline.PipelineVisitor; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.WindowIntoPayload; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.runners.TransformHierarchy.Node; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.PartitioningWindowFn; +import org.apache.beam.sdk.transforms.windowing.Sessions; +import org.apache.beam.sdk.transforms.windowing.SlidingWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.Window.Assign; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; + +/** + * Tests for {@link WindowIntoTranslator}. + */ +@RunWith(Parameterized.class) +public class WindowIntoTranslatorTest { + @Parameters(name = "{index}: {0}") + public static Iterable> data() { + // This pipeline exists for construction, not to run any test. + return ImmutableList.>builder() + .add(FixedWindows.of(Duration.standardMinutes(10L))) + .add(new GlobalWindows()) + .add(Sessions.withGapDuration(Duration.standardMinutes(15L))) + .add(SlidingWindows.of(Duration.standardMinutes(5L)).every(Duration.standardMinutes(1L))) + .add(new CustomWindows()) + .build(); + } + + @Parameter(0) + public WindowFn windowFn; + + @Rule + public TestPipeline pipeline = TestPipeline.create(); + + @Test + public void testToFromProto() throws InvalidProtocolBufferException { + pipeline.apply(GenerateSequence.from(0)).apply(Window.into((WindowFn) windowFn)); + + final AtomicReference>> assign = new AtomicReference<>(null); + pipeline.traverseTopologically( + new PipelineVisitor.Defaults() { + @Override + public void visitPrimitiveTransform(Node node) { + if (node.getTransform() instanceof Window.Assign) { + checkState(assign.get() == null); + assign.set((AppliedPTransform>) node.toAppliedPTransform()); + } + } + }); + checkState(assign.get() != null); + + SdkComponents components = SdkComponents.create(); + WindowIntoPayload payload = + WindowIntoTranslator.toProto(assign.get().getTransform(), components); + + assertEquals(windowFn, WindowIntoTranslator.getWindowFn(payload)); + } + + private static class CustomWindows extends PartitioningWindowFn { + @Override + public BoundedWindow assignWindow(Instant timestamp) { + return GlobalWindow.INSTANCE; + } + + @Override + public boolean isCompatible(WindowFn other) { + return getClass().equals(other.getClass()); + } + + @Override + public Coder windowCoder() { + return (Coder) GlobalWindow.Coder.INSTANCE; + } + + @Override + public boolean equals(Object other) { + return other != null && other.getClass().equals(this.getClass()); + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + } +} From 9da46fd053ef3e7538733d45cacad435c312e51b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 19:02:56 -0700 Subject: [PATCH 033/159] Fix compile error from bad merge --- .../runners/core/construction/WindowIntoTranslatorTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.java index fbac56516cc1..eaefe2eade3b 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.java @@ -84,7 +84,8 @@ public void testToFromProto() throws InvalidProtocolBufferException { public void visitPrimitiveTransform(Node node) { if (node.getTransform() instanceof Window.Assign) { checkState(assign.get() == null); - assign.set((AppliedPTransform>) node.toAppliedPTransform()); + assign.set( + (AppliedPTransform>) node.toAppliedPTransform(getPipeline())); } } }); From 3c0f599d64a7f57608f1c18b05f2ab036a8b02fc Mon Sep 17 00:00:00 2001 From: Colin Phipps Date: Wed, 10 May 2017 09:50:56 +0000 Subject: [PATCH 034/159] Comply with byte limit for Datastore Commit. --- .../io/gcp/datastore/v1/datastoreio.py | 15 ++++++++++++++- .../io/gcp/datastore/v1/datastoreio_test.py | 17 +++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py index c606133b1ece..89c2a93d2780 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py @@ -313,8 +313,12 @@ class _Mutate(PTransform): supported, as the commits are retried when failures occur. """ - # Max allowed Datastore write batch size. + # Max allowed Datastore writes per batch, and max bytes per batch. + # Note that the max bytes per batch set here is lower than the 10MB limit + # actually enforced by the API, to leave space for the CommitRequest wrapper + # around the mutations. _WRITE_BATCH_SIZE = 500 + _WRITE_BATCH_BYTES_SIZE = 9000000 def __init__(self, project, mutation_fn): """Initializes a Mutate transform. @@ -353,13 +357,20 @@ def __init__(self, project): self._project = project self._datastore = None self._mutations = [] + self._mutations_size = 0 # Total size of entries in _mutations. def start_bundle(self): self._mutations = [] + self._mutations_size = 0 self._datastore = helper.get_datastore(self._project) def process(self, element): + size = element.ByteSize() + if (self._mutations and + size + self._mutations_size > _Mutate._WRITE_BATCH_BYTES_SIZE): + self._flush_batch() self._mutations.append(element) + self._mutations_size += size if len(self._mutations) >= _Mutate._WRITE_BATCH_SIZE: self._flush_batch() @@ -367,12 +378,14 @@ def finish_bundle(self): if self._mutations: self._flush_batch() self._mutations = [] + self._mutations_size = 0 def _flush_batch(self): # Flush the current batch of mutations to Cloud Datastore. helper.write_mutations(self._datastore, self._project, self._mutations) logging.debug("Successfully wrote %d mutations.", len(self._mutations)) self._mutations = [] + self._mutations_size = 0 class WriteToDatastore(_Mutate): diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py index 6adc08ad2848..424e714c5737 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py @@ -15,6 +15,7 @@ # limitations under the License. # +import math import unittest from mock import MagicMock, call, patch @@ -191,6 +192,22 @@ def check_DatastoreWriteFn(self, num_entities): self.assertEqual((num_entities - 1) / _Mutate._WRITE_BATCH_SIZE + 1, self._mock_datastore.commit.call_count) + def test_DatastoreWriteLargeEntities(self): + """100*100kB entities gets split over two Commit RPCs.""" + with patch.object(helper, 'get_datastore', + return_value=self._mock_datastore): + entities = [e.entity for e in fake_datastore.create_entities(100)] + + datastore_write_fn = _Mutate.DatastoreWriteFn(self._PROJECT) + datastore_write_fn.start_bundle() + for entity in entities: + datastore_helper.add_properties( + entity, {'large': u'A' * 100000}, exclude_from_indexes=True) + datastore_write_fn.process(WriteToDatastore.to_upsert_mutation(entity)) + datastore_write_fn.finish_bundle() + + self.assertEqual(2, self._mock_datastore.commit.call_count) + def verify_unique_keys(self, queries): """A helper function that verifies if all the queries have unique keys.""" keys, _ = zip(*queries) From 6ed15278650c326cf58ec1f2aa37855dbfc204eb Mon Sep 17 00:00:00 2001 From: Jason Kuster Date: Mon, 22 May 2017 17:46:49 -0700 Subject: [PATCH 035/159] Re-rename fat jar so that install doesn't install the bundled jar as the default jar. Signed-off-by: Jason Kuster --- .../src/main/resources/archetype-resources/pom.xml | 1 + .../examples/src/main/resources/archetype-resources/pom.xml | 1 + 2 files changed, 2 insertions(+) diff --git a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml index 47dfc91face3..af4fbd3832e3 100644 --- a/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/pom.xml @@ -114,6 +114,7 @@ shade + ${project.artifactId}-bundled-${project.version} *:* diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml index 025e3565a21c..b8b9c9f0fa49 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml @@ -113,6 +113,7 @@ shade + ${project.artifactId}-bundled-${project.version} *:* From 94e619d4cbe90ee5d29af346c978d46ab567c951 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 20:27:52 -0700 Subject: [PATCH 036/159] Revert "Include Additional PTransform inputs in Transform Nodes" This reverts commit 9336230d2a5c18bae89908bcd60db8ea96b7906d. It appears to break the BigQueryTornadoesIT. --- .../apex/translation/TranslationContext.java | 4 +- .../core/construction/TransformInputs.java | 50 ------ .../construction/TransformInputsTest.java | 166 ------------------ .../runners/direct/DirectGraphVisitor.java | 15 +- .../runners/direct/ParDoEvaluatorFactory.java | 9 +- ...ttableProcessElementsEvaluatorFactory.java | 2 - .../direct/StatefulParDoEvaluatorFactory.java | 1 - .../beam/runners/direct/WatermarkManager.java | 17 +- .../runners/direct/ParDoEvaluatorTest.java | 6 +- .../flink/FlinkBatchTranslationContext.java | 3 +- .../FlinkStreamingTranslationContext.java | 3 +- .../spark/translation/EvaluationContext.java | 4 +- .../beam/sdk/runners/TransformHierarchy.java | 28 +-- 13 files changed, 28 insertions(+), 280 deletions(-) delete mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java delete mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java index 94d13e177dec..aff3863624c4 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java @@ -34,7 +34,6 @@ import org.apache.beam.runners.apex.translation.utils.ApexStateInternals.ApexStateBackend; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translation.utils.CoderAdapterStreamCodec; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -94,8 +93,7 @@ public Map, PValue> getInputs() { } public InputT getInput() { - return (InputT) - Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(getCurrentTransform())); + return (InputT) Iterables.getOnlyElement(getCurrentTransform().getInputs().values()); } public Map, PValue> getOutputs() { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java deleted file mode 100644 index 2baf93a3c128..000000000000 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.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.runners.core.construction; - -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.common.collect.ImmutableList; -import java.util.Collection; -import java.util.Map; -import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.TupleTag; - -/** Utilities for extracting subsets of inputs from an {@link AppliedPTransform}. */ -public class TransformInputs { - /** - * Gets all inputs of the {@link AppliedPTransform} that are not returned by {@link - * PTransform#getAdditionalInputs()}. - */ - public static Collection nonAdditionalInputs(AppliedPTransform application) { - ImmutableList.Builder mainInputs = ImmutableList.builder(); - PTransform transform = application.getTransform(); - for (Map.Entry, PValue> input : application.getInputs().entrySet()) { - if (!transform.getAdditionalInputs().containsKey(input.getKey())) { - mainInputs.add(input.getValue()); - } - } - checkArgument( - !mainInputs.build().isEmpty() || application.getInputs().isEmpty(), - "Expected at least one main input if any inputs exist"); - return mainInputs.build(); - } -} diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java deleted file mode 100644 index f5b2c11e7923..000000000000 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.core.construction; - -import static org.junit.Assert.assertThat; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.GenerateSequence; -import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.PTransform; -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.POutput; -import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.TupleTag; -import org.hamcrest.Matchers; -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 TransformInputs}. */ -@RunWith(JUnit4.class) -public class TransformInputsTest { - @Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false); - @Rule public ExpectedException thrown = ExpectedException.none(); - - @Test - public void nonAdditionalInputsWithNoInputSucceeds() { - AppliedPTransform transform = - AppliedPTransform.of( - "input-free", - Collections., PValue>emptyMap(), - Collections., PValue>emptyMap(), - new TestTransform(), - pipeline); - - assertThat(TransformInputs.nonAdditionalInputs(transform), Matchers.empty()); - } - - @Test - public void nonAdditionalInputsWithOneMainInputSucceeds() { - PCollection input = pipeline.apply(GenerateSequence.from(1L)); - AppliedPTransform transform = - AppliedPTransform.of( - "input-single", - Collections., PValue>singletonMap(new TupleTag() {}, input), - Collections., PValue>emptyMap(), - new TestTransform(), - pipeline); - - assertThat( - TransformInputs.nonAdditionalInputs(transform), Matchers.containsInAnyOrder(input)); - } - - @Test - public void nonAdditionalInputsWithMultipleNonAdditionalInputsSucceeds() { - Map, PValue> allInputs = new HashMap<>(); - PCollection mainInts = pipeline.apply("MainInput", Create.of(12, 3)); - allInputs.put(new TupleTag() {}, mainInts); - PCollection voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of())); - allInputs.put(new TupleTag() {}, voids); - AppliedPTransform transform = - AppliedPTransform.of( - "additional-free", - allInputs, - Collections., PValue>emptyMap(), - new TestTransform(), - pipeline); - - assertThat( - TransformInputs.nonAdditionalInputs(transform), - Matchers.containsInAnyOrder(voids, mainInts)); - } - - @Test - public void nonAdditionalInputsWithAdditionalInputsSucceeds() { - Map, PValue> additionalInputs = new HashMap<>(); - additionalInputs.put(new TupleTag() {}, pipeline.apply(Create.of("1, 2", "3"))); - additionalInputs.put(new TupleTag() {}, pipeline.apply(GenerateSequence.from(3L))); - - Map, PValue> allInputs = new HashMap<>(); - PCollection mainInts = pipeline.apply("MainInput", Create.of(12, 3)); - allInputs.put(new TupleTag() {}, mainInts); - PCollection voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of())); - allInputs.put( - new TupleTag() {}, voids); - allInputs.putAll(additionalInputs); - - AppliedPTransform transform = - AppliedPTransform.of( - "additional", - allInputs, - Collections., PValue>emptyMap(), - new TestTransform(additionalInputs), - pipeline); - - assertThat( - TransformInputs.nonAdditionalInputs(transform), - Matchers.containsInAnyOrder(mainInts, voids)); - } - - @Test - public void nonAdditionalInputsWithOnlyAdditionalInputsThrows() { - Map, PValue> additionalInputs = new HashMap<>(); - additionalInputs.put(new TupleTag() {}, pipeline.apply(Create.of("1, 2", "3"))); - additionalInputs.put(new TupleTag() {}, pipeline.apply(GenerateSequence.from(3L))); - - AppliedPTransform transform = - AppliedPTransform.of( - "additional-only", - additionalInputs, - Collections., PValue>emptyMap(), - new TestTransform(additionalInputs), - pipeline); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("at least one"); - TransformInputs.nonAdditionalInputs(transform); - } - - private static class TestTransform extends PTransform { - private final Map, PValue> additionalInputs; - - private TestTransform() { - this(Collections., PValue>emptyMap()); - } - - private TestTransform(Map, PValue> additionalInputs) { - this.additionalInputs = additionalInputs; - } - - @Override - public POutput expand(PInput input) { - return PDone.in(input.getPipeline()); - } - - @Override - public Map, PValue> getAdditionalInputs() { - return additionalInputs; - } - } -} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index ed4282bbdb8b..01204e3049dd 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -21,12 +21,10 @@ import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ListMultimap; -import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -36,8 +34,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the @@ -45,7 +41,6 @@ * input after the upstream transform has produced and committed output. */ class DirectGraphVisitor extends PipelineVisitor.Defaults { - private static final Logger LOG = LoggerFactory.getLogger(DirectGraphVisitor.class); private Map> producers = new HashMap<>(); @@ -88,15 +83,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (node.getInputs().isEmpty()) { rootTransforms.add(appliedTransform); } else { - Collection mainInputs = - TransformInputs.nonAdditionalInputs(node.toAppliedPTransform(getPipeline())); - if (!mainInputs.containsAll(node.getInputs().values())) { - LOG.debug( - "Inputs reduced to {} from {} by removing additional inputs", - mainInputs, - node.getInputs().values()); - } - for (PValue value : mainInputs) { + for (PValue value : node.getInputs().values()) { primitiveConsumers.put(value, appliedTransform); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index c52091e27cfe..74470bfb8b8d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -20,6 +20,7 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -78,7 +79,6 @@ public TransformEvaluator forApplication( (TransformEvaluator) createEvaluator( (AppliedPTransform) application, - (PCollection) inputBundle.getPCollection(), inputBundle.getKey(), doFn, transform.getSideInputs(), @@ -102,7 +102,6 @@ public void cleanup() throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( AppliedPTransform, PCollectionTuple, ?> application, - PCollection mainInput, StructuralKey inputBundleKey, DoFn doFn, List> sideInputs, @@ -121,7 +120,6 @@ DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( createParDoEvaluator( application, inputBundleKey, - mainInput, sideInputs, mainOutputTag, additionalOutputTags, @@ -134,7 +132,6 @@ DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( ParDoEvaluator createParDoEvaluator( AppliedPTransform, PCollectionTuple, ?> application, StructuralKey key, - PCollection mainInput, List> sideInputs, TupleTag mainOutputTag, List> additionalOutputTags, @@ -147,7 +144,8 @@ ParDoEvaluator createParDoEvaluator( evaluationContext, stepContext, application, - mainInput.getWindowingStrategy(), + ((PCollection) Iterables.getOnlyElement(application.getInputs().values())) + .getWindowingStrategy(), fn, key, sideInputs, @@ -175,4 +173,5 @@ static Map, PCollection> pcollections(Map, PValue> ou } return pcs; } + } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index 4e7f4db65478..dc85d87bc93f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -116,8 +116,6 @@ public void cleanup() throws Exception { delegateFactory.createParDoEvaluator( application, inputBundle.getKey(), - (PCollection>>) - inputBundle.getPCollection(), transform.getSideInputs(), transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index e22edd187c53..985c3be4e9e9 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -117,7 +117,6 @@ private TransformEvaluator>> createEvaluator( DoFnLifecycleManagerRemovingTransformEvaluator> delegateEvaluator = delegateFactory.createEvaluator( (AppliedPTransform) application, - (PCollection) inputBundle.getPCollection(), inputBundle.getKey(), doFn, application.getTransform().getUnderlyingParDo().getSideInputs(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index b15b52e314de..4f1b8319dc2d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -823,11 +823,10 @@ private Collection getInputProcessingWatermarks(AppliedPTransform getInputWatermarks(AppliedPTransform transform) inputWatermarksBuilder.add(THE_END_OF_TIME); } for (PValue pvalue : inputs.values()) { - if (graph.getPrimitiveConsumers(pvalue).contains(transform)) { - Watermark producerOutputWatermark = - getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; - inputWatermarksBuilder.add(producerOutputWatermark); - } + Watermark producerOutputWatermark = + getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; + inputWatermarksBuilder.add(producerOutputWatermark); } List inputCollectionWatermarks = inputWatermarksBuilder.build(); return inputCollectionWatermarks; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index 3b2a22ee26f9..286e44d1be04 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -98,7 +98,7 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { when(evaluationContext.createBundle(output)).thenReturn(outputBundle); ParDoEvaluator evaluator = - createEvaluator(singletonView, fn, inputPc, output); + createEvaluator(singletonView, fn, output); IntervalWindow nonGlobalWindow = new IntervalWindow(new Instant(0), new Instant(10_000L)); WindowedValue first = WindowedValue.valueInGlobalWindow(3); @@ -132,7 +132,6 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { private ParDoEvaluator createEvaluator( PCollectionView singletonView, RecorderFn fn, - PCollection input, PCollection output) { when( evaluationContext.createSideInputReader( @@ -157,7 +156,8 @@ private ParDoEvaluator createEvaluator( evaluationContext, stepContext, transform, - input.getWindowingStrategy(), + ((PCollection) Iterables.getOnlyElement(transform.getInputs().values())) + .getWindowingStrategy(), fn, null /* key */, ImmutableList.>of(singletonView), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java index 6e7019848b19..0439119dfc40 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java @@ -20,7 +20,6 @@ import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.Map; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -144,7 +143,7 @@ Map, PValue> getInputs(PTransform transform) { @SuppressWarnings("unchecked") T getInput(PTransform transform) { - return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform)); + return (T) Iterables.getOnlyElement(currentTransform.getInputs().values()); } Map, PValue> getOutputs(PTransform transform) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java index 74a5fb971144..ea5f6b3162af 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java @@ -22,7 +22,6 @@ import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.Map; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -114,7 +113,7 @@ public TypeInformation> getTypeInfo(PCollection collecti @SuppressWarnings("unchecked") public T getInput(PTransform transform) { - return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform)); + return (T) Iterables.getOnlyElement(currentTransform.getInputs().values()); } public Map, PValue> getInputs(PTransform transform) { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index 0c6c4d1cb660..8102926f6daa 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -26,7 +26,6 @@ import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.sdk.Pipeline; @@ -104,8 +103,7 @@ public void setCurrentTransform(AppliedPTransform transform) { public T getInput(PTransform transform) { @SuppressWarnings("unchecked") - T input = - (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(getCurrentTransform())); + T input = (T) Iterables.getOnlyElement(getInputs(transform).values()); return input; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index 9d73b4576788..2f0e8efd7de8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -32,6 +32,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; @@ -67,7 +68,7 @@ public TransformHierarchy() { producers = new HashMap<>(); producerInput = new HashMap<>(); unexpandedInputs = new HashMap<>(); - root = new Node(); + root = new Node(null, null, "", null); current = root; } @@ -251,37 +252,26 @@ public class Node { @VisibleForTesting boolean finishedSpecifying = false; - /** - * Creates the root-level node. The root level node has a null enclosing node, a null transform, - * an empty map of inputs, and a name equal to the empty string. - */ - private Node() { - this.enclosingNode = null; - this.transform = null; - this.fullName = ""; - this.inputs = Collections.emptyMap(); - } - /** * Creates a new Node with the given parent and transform. * + *

EnclosingNode and transform may both be null for a root-level node, which holds all other + * nodes. + * * @param enclosingNode the composite node containing this node * @param transform the PTransform tracked by this node * @param fullName the fully qualified name of the transform * @param input the unexpanded input to the transform */ private Node( - Node enclosingNode, - PTransform transform, + @Nullable Node enclosingNode, + @Nullable PTransform transform, String fullName, - PInput input) { + @Nullable PInput input) { this.enclosingNode = enclosingNode; this.transform = transform; this.fullName = fullName; - ImmutableMap.Builder, PValue> inputs = ImmutableMap.builder(); - inputs.putAll(input.expand()); - inputs.putAll(transform.getAdditionalInputs()); - this.inputs = inputs.build(); + this.inputs = input == null ? Collections., PValue>emptyMap() : input.expand(); } /** From 3a62b4f7b20bda2b3c4ca648f90988d387cfe20d Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 18 May 2017 17:22:25 -0700 Subject: [PATCH 037/159] Automatically convert examples to use with syntax. --- .../examples/complete/autocomplete.py | 19 +- .../examples/complete/autocomplete_test.py | 31 +- .../examples/complete/estimate_pi.py | 11 +- .../examples/complete/estimate_pi_test.py | 12 +- .../complete/game/hourly_team_score.py | 19 +- .../examples/complete/game/user_score.py | 15 +- .../complete/juliaset/juliaset/juliaset.py | 44 +- .../apache_beam/examples/complete/tfidf.py | 21 +- .../examples/complete/tfidf_test.py | 28 +- .../complete/top_wikipedia_sessions.py | 12 +- .../complete/top_wikipedia_sessions_test.py | 9 +- .../examples/cookbook/bigquery_schema.py | 159 +++--- .../examples/cookbook/bigquery_side_input.py | 51 +- .../cookbook/bigquery_side_input_test.py | 39 +- .../examples/cookbook/bigquery_tornadoes.py | 33 +- .../cookbook/bigquery_tornadoes_test.py | 19 +- .../apache_beam/examples/cookbook/coders.py | 16 +- .../examples/cookbook/coders_test.py | 14 +- .../examples/cookbook/custom_ptransform.py | 27 +- .../cookbook/custom_ptransform_test.py | 11 +- .../examples/cookbook/datastore_wordcount.py | 20 +- .../apache_beam/examples/cookbook/filters.py | 21 +- .../examples/cookbook/group_with_coder.py | 43 +- .../cookbook/group_with_coder_test.py | 4 +- .../examples/cookbook/mergecontacts.py | 115 ++-- .../examples/cookbook/mergecontacts_test.py | 3 +- .../cookbook/multiple_output_pardo.py | 72 ++- .../cookbook/multiple_output_pardo_test.py | 2 +- .../apache_beam/examples/snippets/snippets.py | 494 +++++++++--------- .../examples/snippets/snippets_test.py | 326 ++++++------ .../apache_beam/examples/streaming_wordcap.py | 24 +- .../examples/streaming_wordcount.py | 44 +- sdks/python/apache_beam/examples/wordcount.py | 1 - .../examples/wordcount_debugging.py | 55 +- .../apache_beam/examples/wordcount_minimal.py | 33 +- .../apache_beam/io/filebasedsink_test.py | 16 +- sdks/python/apache_beam/pipeline.py | 19 +- .../apache_beam/transforms/combiners_test.py | 58 +- .../apache_beam/transforms/window_test.py | 147 +++--- .../transforms/write_ptransform_test.py | 7 +- .../typehints/typed_pipeline_test.py | 22 +- 41 files changed, 1005 insertions(+), 1111 deletions(-) diff --git a/sdks/python/apache_beam/examples/complete/autocomplete.py b/sdks/python/apache_beam/examples/complete/autocomplete.py index f0acc3fc562d..ab3397cfe335 100644 --- a/sdks/python/apache_beam/examples/complete/autocomplete.py +++ b/sdks/python/apache_beam/examples/complete/autocomplete.py @@ -44,16 +44,15 @@ def run(argv=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) - - (p # pylint: disable=expression-not-assigned - | 'read' >> ReadFromText(known_args.input) - | 'split' >> beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) - | 'TopPerPrefix' >> TopPerPrefix(5) - | 'format' >> beam.Map( - lambda (prefix, candidates): '%s: %s' % (prefix, candidates)) - | 'write' >> WriteToText(known_args.output)) - p.run() + with beam.Pipeline(options=pipeline_options) as p: + + (p # pylint: disable=expression-not-assigned + | 'read' >> ReadFromText(known_args.input) + | 'split' >> beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) + | 'TopPerPrefix' >> TopPerPrefix(5) + | 'format' >> beam.Map( + lambda (prefix, candidates): '%s: %s' % (prefix, candidates)) + | 'write' >> WriteToText(known_args.output)) class TopPerPrefix(beam.PTransform): diff --git a/sdks/python/apache_beam/examples/complete/autocomplete_test.py b/sdks/python/apache_beam/examples/complete/autocomplete_test.py index 378d222bfa8d..e2c84d68d3d4 100644 --- a/sdks/python/apache_beam/examples/complete/autocomplete_test.py +++ b/sdks/python/apache_beam/examples/complete/autocomplete_test.py @@ -31,22 +31,21 @@ class AutocompleteTest(unittest.TestCase): WORDS = ['this', 'this', 'that', 'to', 'to', 'to'] def test_top_prefixes(self): - p = TestPipeline() - words = p | beam.Create(self.WORDS) - result = words | autocomplete.TopPerPrefix(5) - # values must be hashable for now - result = result | beam.Map(lambda (k, vs): (k, tuple(vs))) - assert_that(result, equal_to( - [ - ('t', ((3, 'to'), (2, 'this'), (1, 'that'))), - ('to', ((3, 'to'), )), - ('th', ((2, 'this'), (1, 'that'))), - ('thi', ((2, 'this'), )), - ('this', ((2, 'this'), )), - ('tha', ((1, 'that'), )), - ('that', ((1, 'that'), )), - ])) - p.run() + with TestPipeline() as p: + words = p | beam.Create(self.WORDS) + result = words | autocomplete.TopPerPrefix(5) + # values must be hashable for now + result = result | beam.Map(lambda (k, vs): (k, tuple(vs))) + assert_that(result, equal_to( + [ + ('t', ((3, 'to'), (2, 'this'), (1, 'that'))), + ('to', ((3, 'to'), )), + ('th', ((2, 'this'), (1, 'that'))), + ('thi', ((2, 'this'), )), + ('this', ((2, 'this'), )), + ('tha', ((1, 'that'), )), + ('that', ((1, 'that'), )), + ])) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/complete/estimate_pi.py b/sdks/python/apache_beam/examples/complete/estimate_pi.py index c709713bd97e..7e3c4cd35a27 100644 --- a/sdks/python/apache_beam/examples/complete/estimate_pi.py +++ b/sdks/python/apache_beam/examples/complete/estimate_pi.py @@ -113,14 +113,11 @@ def run(argv=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) + with beam.Pipeline(options=pipeline_options) as p: - (p # pylint: disable=expression-not-assigned - | EstimatePiTransform() - | WriteToText(known_args.output, coder=JsonCoder())) - - # Actually run the pipeline (all operations above are deferred). - p.run() + (p # pylint: disable=expression-not-assigned + | EstimatePiTransform() + | WriteToText(known_args.output, coder=JsonCoder())) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/complete/estimate_pi_test.py b/sdks/python/apache_beam/examples/complete/estimate_pi_test.py index fd5130966887..f1cbb0a24d56 100644 --- a/sdks/python/apache_beam/examples/complete/estimate_pi_test.py +++ b/sdks/python/apache_beam/examples/complete/estimate_pi_test.py @@ -38,13 +38,13 @@ def _in_between(actual): class EstimatePiTest(unittest.TestCase): def test_basics(self): - p = TestPipeline() - result = p | 'Estimate' >> estimate_pi.EstimatePiTransform(5000) + with TestPipeline() as p: + result = p | 'Estimate' >> estimate_pi.EstimatePiTransform(5000) - # Note: Probabilistically speaking this test can fail with a probability - # that is very small (VERY) given that we run at least 500 thousand trials. - assert_that(result, in_between(3.125, 3.155)) - p.run() + # Note: Probabilistically speaking this test can fail with a probability + # that is very small (VERY) given that we run at least 500 thousand + # trials. + assert_that(result, in_between(3.125, 3.155)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py b/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py index e9d71881f8c9..9f398d9995f2 100644 --- a/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py +++ b/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py @@ -276,18 +276,15 @@ def run(argv=None): known_args, pipeline_args = parser.parse_known_args(argv) pipeline_options = PipelineOptions(pipeline_args) - p = beam.Pipeline(options=pipeline_options) pipeline_options.view_as(SetupOptions).save_main_session = True - - (p # pylint: disable=expression-not-assigned - | ReadFromText(known_args.input) - | HourlyTeamScore( - known_args.start_min, known_args.stop_min, known_args.window_duration) - | WriteWindowedToBigQuery( - known_args.table_name, known_args.dataset, configure_bigquery_write())) - - result = p.run() - result.wait_until_finish() + with beam.Pipeline(options=pipeline_options) as p: + + (p # pylint: disable=expression-not-assigned + | ReadFromText(known_args.input) + | HourlyTeamScore( + known_args.start_min, known_args.stop_min, known_args.window_duration) + | WriteWindowedToBigQuery( + known_args.table_name, known_args.dataset, configure_bigquery_write())) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/complete/game/user_score.py b/sdks/python/apache_beam/examples/complete/game/user_score.py index 389d2c6a8433..c9f273814a85 100644 --- a/sdks/python/apache_beam/examples/complete/game/user_score.py +++ b/sdks/python/apache_beam/examples/complete/game/user_score.py @@ -201,16 +201,13 @@ def run(argv=None): known_args, pipeline_args = parser.parse_known_args(argv) pipeline_options = PipelineOptions(pipeline_args) - p = beam.Pipeline(options=pipeline_options) + with beam.Pipeline(options=pipeline_options) as p: - (p # pylint: disable=expression-not-assigned - | ReadFromText(known_args.input) # Read events from a file and parse them. - | UserScore() - | WriteToBigQuery( - known_args.table_name, known_args.dataset, configure_bigquery_write())) - - result = p.run() - result.wait_until_finish() + (p # pylint: disable=expression-not-assigned + | ReadFromText(known_args.input) # Read events from a file and parse them. + | UserScore() + | WriteToBigQuery( + known_args.table_name, known_args.dataset, configure_bigquery_write())) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset.py b/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset.py index 5ff2b785108b..61e3fd1a8d0c 100644 --- a/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset.py +++ b/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset.py @@ -99,26 +99,24 @@ def run(argv=None): # pylint: disable=missing-docstring help='Output file to write the resulting image to.') known_args, pipeline_args = parser.parse_known_args(argv) - p = beam.Pipeline(argv=pipeline_args) - n = int(known_args.grid_size) - - coordinates = generate_julia_set_colors(p, complex(-.62772, .42193), n, 100) - - # Group each coordinate triplet by its x value, then write the coordinates to - # the output file with an x-coordinate grouping per line. - # pylint: disable=expression-not-assigned - (coordinates - | 'x coord key' >> beam.Map(lambda (x, y, i): (x, (x, y, i))) - | 'x coord' >> beam.GroupByKey() - | 'format' >> beam.Map( - lambda (k, coords): ' '.join('(%s, %s, %s)' % coord for coord in coords)) - | WriteToText(known_args.coordinate_output)) - # pylint: enable=expression-not-assigned - return p.run().wait_until_finish() - - # Optionally render the image and save it to a file. - # TODO(silviuc): Add this functionality. - # if p.options.image_output is not None: - # julia_set_image = generate_julia_set_visualization( - # file_with_coordinates, n, 100) - # save_julia_set_visualization(p.options.image_output, julia_set_image) + with beam.Pipeline(argv=pipeline_args) as p: + n = int(known_args.grid_size) + + coordinates = generate_julia_set_colors(p, complex(-.62772, .42193), n, 100) + + # Group each coordinate triplet by its x value, then write the coordinates + # to the output file with an x-coordinate grouping per line. + # pylint: disable=expression-not-assigned + (coordinates + | 'x coord key' >> beam.Map(lambda (x, y, i): (x, (x, y, i))) + | 'x coord' >> beam.GroupByKey() + | 'format' >> beam.Map( + lambda (k, coords): ' '.join('(%s, %s, %s)' % c for c in coords)) + | WriteToText(known_args.coordinate_output)) + + # Optionally render the image and save it to a file. + # TODO(silviuc): Add this functionality. + # if p.options.image_output is not None: + # julia_set_image = generate_julia_set_visualization( + # file_with_coordinates, n, 100) + # save_julia_set_visualization(p.options.image_output, julia_set_image) diff --git a/sdks/python/apache_beam/examples/complete/tfidf.py b/sdks/python/apache_beam/examples/complete/tfidf.py index a98d90640d52..a88ff827766c 100644 --- a/sdks/python/apache_beam/examples/complete/tfidf.py +++ b/sdks/python/apache_beam/examples/complete/tfidf.py @@ -191,17 +191,16 @@ def run(argv=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) - - # Read documents specified by the uris command line option. - pcoll = read_documents(p, glob.glob(known_args.uris)) - # Compute TF-IDF information for each word. - output = pcoll | TfIdf() - # Write the output using a "Write" transform that has side effects. - # pylint: disable=expression-not-assigned - output | 'write' >> WriteToText(known_args.output) - # Execute the pipeline and wait until it is completed. - p.run().wait_until_finish() + with beam.Pipeline(options=pipeline_options) as p: + + # Read documents specified by the uris command line option. + pcoll = read_documents(p, glob.glob(known_args.uris)) + # Compute TF-IDF information for each word. + output = pcoll | TfIdf() + # Write the output using a "Write" transform that has side effects. + # pylint: disable=expression-not-assigned + output | 'write' >> WriteToText(known_args.output) + # Execute the pipeline and wait until it is completed. if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/complete/tfidf_test.py b/sdks/python/apache_beam/examples/complete/tfidf_test.py index f177dfc9e166..322426fd2b3d 100644 --- a/sdks/python/apache_beam/examples/complete/tfidf_test.py +++ b/sdks/python/apache_beam/examples/complete/tfidf_test.py @@ -50,20 +50,20 @@ def create_file(self, path, contents): f.write(contents) def test_tfidf_transform(self): - p = TestPipeline() - uri_to_line = p | 'create sample' >> beam.Create( - [('1.txt', 'abc def ghi'), - ('2.txt', 'abc def'), - ('3.txt', 'abc')]) - result = ( - uri_to_line - | tfidf.TfIdf() - | beam.Map(lambda (word, (uri, tfidf)): (word, uri, tfidf))) - assert_that(result, equal_to(EXPECTED_RESULTS)) - # Run the pipeline. Note that the assert_that above adds to the pipeline - # a check that the result PCollection contains expected values. To actually - # trigger the check the pipeline must be run. - p.run() + with TestPipeline() as p: + uri_to_line = p | 'create sample' >> beam.Create( + [('1.txt', 'abc def ghi'), + ('2.txt', 'abc def'), + ('3.txt', 'abc')]) + result = ( + uri_to_line + | tfidf.TfIdf() + | beam.Map(lambda (word, (uri, tfidf)): (word, uri, tfidf))) + assert_that(result, equal_to(EXPECTED_RESULTS)) + # Run the pipeline. Note that the assert_that above adds to the pipeline + # a check that the result PCollection contains expected values. + # To actually trigger the check the pipeline must be run (e.g. by + # exiting the with context). def test_basics(self): # Setup the files with expected content. diff --git a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py index aa48e4eb920e..9a9ad7865c2c 100644 --- a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py +++ b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions.py @@ -159,14 +159,12 @@ def run(argv=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) + with beam.Pipeline(options=pipeline_options) as p: - (p # pylint: disable=expression-not-assigned - | ReadFromText(known_args.input) - | ComputeTopSessions(known_args.sampling_threshold) - | WriteToText(known_args.output)) - - p.run() + (p # pylint: disable=expression-not-assigned + | ReadFromText(known_args.input) + | ComputeTopSessions(known_args.sampling_threshold) + | WriteToText(known_args.output)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py index 5fb6276cb8d5..ced8a44af13c 100644 --- a/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py +++ b/sdks/python/apache_beam/examples/complete/top_wikipedia_sessions_test.py @@ -52,12 +52,11 @@ class ComputeTopSessionsTest(unittest.TestCase): ] def test_compute_top_sessions(self): - p = TestPipeline() - edits = p | beam.Create(self.EDITS) - result = edits | top_wikipedia_sessions.ComputeTopSessions(1.0) + with TestPipeline() as p: + edits = p | beam.Create(self.EDITS) + result = edits | top_wikipedia_sessions.ComputeTopSessions(1.0) - assert_that(result, equal_to(self.EXPECTED)) - p.run() + assert_that(result, equal_to(self.EXPECTED)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_schema.py b/sdks/python/apache_beam/examples/cookbook/bigquery_schema.py index 400189e64424..3a8af67d1391 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_schema.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_schema.py @@ -42,86 +42,85 @@ def run(argv=None): 'or DATASET.TABLE.')) known_args, pipeline_args = parser.parse_known_args(argv) - p = beam.Pipeline(argv=pipeline_args) - - from apache_beam.io.gcp.internal.clients import bigquery # pylint: disable=wrong-import-order, wrong-import-position - - table_schema = bigquery.TableSchema() - - # Fields that use standard types. - kind_schema = bigquery.TableFieldSchema() - kind_schema.name = 'kind' - kind_schema.type = 'string' - kind_schema.mode = 'nullable' - table_schema.fields.append(kind_schema) - - full_name_schema = bigquery.TableFieldSchema() - full_name_schema.name = 'fullName' - full_name_schema.type = 'string' - full_name_schema.mode = 'required' - table_schema.fields.append(full_name_schema) - - age_schema = bigquery.TableFieldSchema() - age_schema.name = 'age' - age_schema.type = 'integer' - age_schema.mode = 'nullable' - table_schema.fields.append(age_schema) - - gender_schema = bigquery.TableFieldSchema() - gender_schema.name = 'gender' - gender_schema.type = 'string' - gender_schema.mode = 'nullable' - table_schema.fields.append(gender_schema) - - # A nested field - phone_number_schema = bigquery.TableFieldSchema() - phone_number_schema.name = 'phoneNumber' - phone_number_schema.type = 'record' - phone_number_schema.mode = 'nullable' - - area_code = bigquery.TableFieldSchema() - area_code.name = 'areaCode' - area_code.type = 'integer' - area_code.mode = 'nullable' - phone_number_schema.fields.append(area_code) - - number = bigquery.TableFieldSchema() - number.name = 'number' - number.type = 'integer' - number.mode = 'nullable' - phone_number_schema.fields.append(number) - table_schema.fields.append(phone_number_schema) - - # A repeated field. - children_schema = bigquery.TableFieldSchema() - children_schema.name = 'children' - children_schema.type = 'string' - children_schema.mode = 'repeated' - table_schema.fields.append(children_schema) - - def create_random_record(record_id): - return {'kind': 'kind' + record_id, 'fullName': 'fullName'+record_id, - 'age': int(record_id) * 10, 'gender': 'male', - 'phoneNumber': { - 'areaCode': int(record_id) * 100, - 'number': int(record_id) * 100000}, - 'children': ['child' + record_id + '1', - 'child' + record_id + '2', - 'child' + record_id + '3'] - } - - # pylint: disable=expression-not-assigned - record_ids = p | 'CreateIDs' >> beam.Create(['1', '2', '3', '4', '5']) - records = record_ids | 'CreateRecords' >> beam.Map(create_random_record) - records | 'write' >> beam.io.Write( - beam.io.BigQuerySink( - known_args.output, - schema=table_schema, - create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, - write_disposition=beam.io.BigQueryDisposition.WRITE_TRUNCATE)) - - # Run the pipeline (all operations are deferred until run() is called). - p.run() + with beam.Pipeline(argv=pipeline_args) as p: + + from apache_beam.io.gcp.internal.clients import bigquery # pylint: disable=wrong-import-order, wrong-import-position + + table_schema = bigquery.TableSchema() + + # Fields that use standard types. + kind_schema = bigquery.TableFieldSchema() + kind_schema.name = 'kind' + kind_schema.type = 'string' + kind_schema.mode = 'nullable' + table_schema.fields.append(kind_schema) + + full_name_schema = bigquery.TableFieldSchema() + full_name_schema.name = 'fullName' + full_name_schema.type = 'string' + full_name_schema.mode = 'required' + table_schema.fields.append(full_name_schema) + + age_schema = bigquery.TableFieldSchema() + age_schema.name = 'age' + age_schema.type = 'integer' + age_schema.mode = 'nullable' + table_schema.fields.append(age_schema) + + gender_schema = bigquery.TableFieldSchema() + gender_schema.name = 'gender' + gender_schema.type = 'string' + gender_schema.mode = 'nullable' + table_schema.fields.append(gender_schema) + + # A nested field + phone_number_schema = bigquery.TableFieldSchema() + phone_number_schema.name = 'phoneNumber' + phone_number_schema.type = 'record' + phone_number_schema.mode = 'nullable' + + area_code = bigquery.TableFieldSchema() + area_code.name = 'areaCode' + area_code.type = 'integer' + area_code.mode = 'nullable' + phone_number_schema.fields.append(area_code) + + number = bigquery.TableFieldSchema() + number.name = 'number' + number.type = 'integer' + number.mode = 'nullable' + phone_number_schema.fields.append(number) + table_schema.fields.append(phone_number_schema) + + # A repeated field. + children_schema = bigquery.TableFieldSchema() + children_schema.name = 'children' + children_schema.type = 'string' + children_schema.mode = 'repeated' + table_schema.fields.append(children_schema) + + def create_random_record(record_id): + return {'kind': 'kind' + record_id, 'fullName': 'fullName'+record_id, + 'age': int(record_id) * 10, 'gender': 'male', + 'phoneNumber': { + 'areaCode': int(record_id) * 100, + 'number': int(record_id) * 100000}, + 'children': ['child' + record_id + '1', + 'child' + record_id + '2', + 'child' + record_id + '3'] + } + + # pylint: disable=expression-not-assigned + record_ids = p | 'CreateIDs' >> beam.Create(['1', '2', '3', '4', '5']) + records = record_ids | 'CreateRecords' >> beam.Map(create_random_record) + records | 'write' >> beam.io.Write( + beam.io.BigQuerySink( + known_args.output, + schema=table_schema, + create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, + write_disposition=beam.io.BigQueryDisposition.WRITE_TRUNCATE)) + + # Run the pipeline (all operations are deferred until run() is called). if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py b/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py index 6b2881828a0d..9911a6716bd8 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py @@ -88,32 +88,31 @@ def run(argv=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) - - group_ids = [] - for i in xrange(0, int(known_args.num_groups)): - group_ids.append('id' + str(i)) - - query_corpus = 'select UNIQUE(corpus) from publicdata:samples.shakespeare' - query_word = 'select UNIQUE(word) from publicdata:samples.shakespeare' - ignore_corpus = known_args.ignore_corpus - ignore_word = known_args.ignore_word - - pcoll_corpus = p | 'read corpus' >> beam.io.Read( - beam.io.BigQuerySource(query=query_corpus)) - pcoll_word = p | 'read_words' >> beam.io.Read( - beam.io.BigQuerySource(query=query_word)) - pcoll_ignore_corpus = p | 'create_ignore_corpus' >> beam.Create( - [ignore_corpus]) - pcoll_ignore_word = p | 'create_ignore_word' >> beam.Create([ignore_word]) - pcoll_group_ids = p | 'create groups' >> beam.Create(group_ids) - - pcoll_groups = create_groups(pcoll_group_ids, pcoll_corpus, pcoll_word, - pcoll_ignore_corpus, pcoll_ignore_word) - - # pylint:disable=expression-not-assigned - pcoll_groups | WriteToText(known_args.output) - p.run() + with beam.Pipeline(options=pipeline_options) as p: + + group_ids = [] + for i in xrange(0, int(known_args.num_groups)): + group_ids.append('id' + str(i)) + + query_corpus = 'select UNIQUE(corpus) from publicdata:samples.shakespeare' + query_word = 'select UNIQUE(word) from publicdata:samples.shakespeare' + ignore_corpus = known_args.ignore_corpus + ignore_word = known_args.ignore_word + + pcoll_corpus = p | 'read corpus' >> beam.io.Read( + beam.io.BigQuerySource(query=query_corpus)) + pcoll_word = p | 'read_words' >> beam.io.Read( + beam.io.BigQuerySource(query=query_word)) + pcoll_ignore_corpus = p | 'create_ignore_corpus' >> beam.Create( + [ignore_corpus]) + pcoll_ignore_word = p | 'create_ignore_word' >> beam.Create([ignore_word]) + pcoll_group_ids = p | 'create groups' >> beam.Create(group_ids) + + pcoll_groups = create_groups(pcoll_group_ids, pcoll_corpus, pcoll_word, + pcoll_ignore_corpus, pcoll_ignore_word) + + # pylint:disable=expression-not-assigned + pcoll_groups | WriteToText(known_args.output) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_side_input_test.py b/sdks/python/apache_beam/examples/cookbook/bigquery_side_input_test.py index b11dc47c3363..964b35b3f08f 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_side_input_test.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_side_input_test.py @@ -30,25 +30,26 @@ class BigQuerySideInputTest(unittest.TestCase): def test_create_groups(self): - p = TestPipeline() - - group_ids_pcoll = p | 'CreateGroupIds' >> beam.Create(['A', 'B', 'C']) - corpus_pcoll = p | 'CreateCorpus' >> beam.Create( - [{'f': 'corpus1'}, {'f': 'corpus2'}, {'f': 'corpus3'}]) - words_pcoll = p | 'CreateWords' >> beam.Create( - [{'f': 'word1'}, {'f': 'word2'}, {'f': 'word3'}]) - ignore_corpus_pcoll = p | 'CreateIgnoreCorpus' >> beam.Create(['corpus1']) - ignore_word_pcoll = p | 'CreateIgnoreWord' >> beam.Create(['word1']) - - groups = bigquery_side_input.create_groups(group_ids_pcoll, corpus_pcoll, - words_pcoll, ignore_corpus_pcoll, - ignore_word_pcoll) - - assert_that(groups, equal_to( - [('A', 'corpus2', 'word2'), - ('B', 'corpus2', 'word2'), - ('C', 'corpus2', 'word2')])) - p.run() + with TestPipeline() as p: + + group_ids_pcoll = p | 'CreateGroupIds' >> beam.Create(['A', 'B', 'C']) + corpus_pcoll = p | 'CreateCorpus' >> beam.Create( + [{'f': 'corpus1'}, {'f': 'corpus2'}, {'f': 'corpus3'}]) + words_pcoll = p | 'CreateWords' >> beam.Create( + [{'f': 'word1'}, {'f': 'word2'}, {'f': 'word3'}]) + ignore_corpus_pcoll = p | 'CreateIgnoreCorpus' >> beam.Create(['corpus1']) + ignore_word_pcoll = p | 'CreateIgnoreWord' >> beam.Create(['word1']) + + groups = bigquery_side_input.create_groups(group_ids_pcoll, + corpus_pcoll, + words_pcoll, + ignore_corpus_pcoll, + ignore_word_pcoll) + + assert_that(groups, equal_to( + [('A', 'corpus2', 'word2'), + ('B', 'corpus2', 'word2'), + ('C', 'corpus2', 'word2')])) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes.py b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes.py index ed0c79a71a15..d3b216e6dd3a 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes.py @@ -75,23 +75,22 @@ def run(argv=None): 'or DATASET.TABLE.')) known_args, pipeline_args = parser.parse_known_args(argv) - p = beam.Pipeline(argv=pipeline_args) - - # Read the table rows into a PCollection. - rows = p | 'read' >> beam.io.Read(beam.io.BigQuerySource(known_args.input)) - counts = count_tornadoes(rows) - - # Write the output using a "Write" transform that has side effects. - # pylint: disable=expression-not-assigned - counts | 'write' >> beam.io.Write( - beam.io.BigQuerySink( - known_args.output, - schema='month:INTEGER, tornado_count:INTEGER', - create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, - write_disposition=beam.io.BigQueryDisposition.WRITE_TRUNCATE)) - - # Run the pipeline (all operations are deferred until run() is called). - p.run().wait_until_finish() + with beam.Pipeline(argv=pipeline_args) as p: + + # Read the table rows into a PCollection. + rows = p | 'read' >> beam.io.Read(beam.io.BigQuerySource(known_args.input)) + counts = count_tornadoes(rows) + + # Write the output using a "Write" transform that has side effects. + # pylint: disable=expression-not-assigned + counts | 'write' >> beam.io.Write( + beam.io.BigQuerySink( + known_args.output, + schema='month:INTEGER, tornado_count:INTEGER', + create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, + write_disposition=beam.io.BigQueryDisposition.WRITE_TRUNCATE)) + + # Run the pipeline (all operations are deferred until run() is called). if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_test.py b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_test.py index c926df8ec15c..45dcabaf853c 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_test.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_test.py @@ -30,16 +30,15 @@ class BigQueryTornadoesTest(unittest.TestCase): def test_basics(self): - p = TestPipeline() - rows = (p | 'create' >> beam.Create([ - {'month': 1, 'day': 1, 'tornado': False}, - {'month': 1, 'day': 2, 'tornado': True}, - {'month': 1, 'day': 3, 'tornado': True}, - {'month': 2, 'day': 1, 'tornado': True}])) - results = bigquery_tornadoes.count_tornadoes(rows) - assert_that(results, equal_to([{'month': 1, 'tornado_count': 2}, - {'month': 2, 'tornado_count': 1}])) - p.run().wait_until_finish() + with TestPipeline() as p: + rows = (p | 'create' >> beam.Create([ + {'month': 1, 'day': 1, 'tornado': False}, + {'month': 1, 'day': 2, 'tornado': True}, + {'month': 1, 'day': 3, 'tornado': True}, + {'month': 2, 'day': 1, 'tornado': True}])) + results = bigquery_tornadoes.count_tornadoes(rows) + assert_that(results, equal_to([{'month': 1, 'tornado_count': 2}, + {'month': 2, 'tornado_count': 1}])) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/coders.py b/sdks/python/apache_beam/examples/cookbook/coders.py index aeeb3c9128a5..f97b0f2d9c9a 100644 --- a/sdks/python/apache_beam/examples/cookbook/coders.py +++ b/sdks/python/apache_beam/examples/cookbook/coders.py @@ -85,15 +85,13 @@ def run(argv=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) - - p = beam.Pipeline(argv=pipeline_args) - (p # pylint: disable=expression-not-assigned - | 'read' >> ReadFromText(known_args.input, coder=JsonCoder()) - | 'points' >> beam.FlatMap(compute_points) - | beam.CombinePerKey(sum) - | 'write' >> WriteToText(known_args.output, coder=JsonCoder())) - p.run() + + with beam.Pipeline(options=pipeline_options) as p: + (p # pylint: disable=expression-not-assigned + | 'read' >> ReadFromText(known_args.input, coder=JsonCoder()) + | 'points' >> beam.FlatMap(compute_points) + | beam.CombinePerKey(sum) + | 'write' >> WriteToText(known_args.output, coder=JsonCoder())) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/coders_test.py b/sdks/python/apache_beam/examples/cookbook/coders_test.py index f71dad8ed80f..988d3c9d25e1 100644 --- a/sdks/python/apache_beam/examples/cookbook/coders_test.py +++ b/sdks/python/apache_beam/examples/cookbook/coders_test.py @@ -35,13 +35,13 @@ class CodersTest(unittest.TestCase): {'host': ['Brasil', 1], 'guest': ['Italy', 0]}] def test_compute_points(self): - p = TestPipeline() - records = p | 'create' >> beam.Create(self.SAMPLE_RECORDS) - result = (records - | 'points' >> beam.FlatMap(coders.compute_points) - | beam.CombinePerKey(sum)) - assert_that(result, equal_to([('Italy', 0), ('Brasil', 6), ('Germany', 3)])) - p.run() + with TestPipeline() as p: + records = p | 'create' >> beam.Create(self.SAMPLE_RECORDS) + result = (records + | 'points' >> beam.FlatMap(coders.compute_points) + | beam.CombinePerKey(sum)) + assert_that(result, + equal_to([('Italy', 0), ('Brasil', 6), ('Germany', 3)])) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/custom_ptransform.py b/sdks/python/apache_beam/examples/cookbook/custom_ptransform.py index 609f2cd87181..aee69d23ff65 100644 --- a/sdks/python/apache_beam/examples/cookbook/custom_ptransform.py +++ b/sdks/python/apache_beam/examples/cookbook/custom_ptransform.py @@ -47,11 +47,10 @@ def expand(self, pcoll): def run_count1(known_args, options): """Runs the first example pipeline.""" logging.info('Running first pipeline') - p = beam.Pipeline(options=options) - (p | beam.io.ReadFromText(known_args.input) - | Count1() - | beam.io.WriteToText(known_args.output)) - p.run().wait_until_finish() + with beam.Pipeline(options=options) as p: + (p | beam.io.ReadFromText(known_args.input) + | Count1() + | beam.io.WriteToText(known_args.output)) @beam.ptransform_fn @@ -66,11 +65,10 @@ def Count2(pcoll): # pylint: disable=invalid-name def run_count2(known_args, options): """Runs the second example pipeline.""" logging.info('Running second pipeline') - p = beam.Pipeline(options=options) - (p | ReadFromText(known_args.input) - | Count2() # pylint: disable=no-value-for-parameter - | WriteToText(known_args.output)) - p.run().wait_until_finish() + with beam.Pipeline(options=options) as p: + (p | ReadFromText(known_args.input) + | Count2() # pylint: disable=no-value-for-parameter + | WriteToText(known_args.output)) @beam.ptransform_fn @@ -93,11 +91,10 @@ def Count3(pcoll, factor=1): # pylint: disable=invalid-name def run_count3(known_args, options): """Runs the third example pipeline.""" logging.info('Running third pipeline') - p = beam.Pipeline(options=options) - (p | ReadFromText(known_args.input) - | Count3(2) # pylint: disable=no-value-for-parameter - | WriteToText(known_args.output)) - p.run() + with beam.Pipeline(options=options) as p: + (p | ReadFromText(known_args.input) + | Count3(2) # pylint: disable=no-value-for-parameter + | WriteToText(known_args.output)) def get_args(argv): diff --git a/sdks/python/apache_beam/examples/cookbook/custom_ptransform_test.py b/sdks/python/apache_beam/examples/cookbook/custom_ptransform_test.py index c7c6dbabd5e0..7aaccb4ac1c0 100644 --- a/sdks/python/apache_beam/examples/cookbook/custom_ptransform_test.py +++ b/sdks/python/apache_beam/examples/cookbook/custom_ptransform_test.py @@ -40,12 +40,11 @@ def test_count3(self): self.run_pipeline(custom_ptransform.Count3(factor), factor=factor) def run_pipeline(self, count_implementation, factor=1): - p = TestPipeline() - words = p | beam.Create(['CAT', 'DOG', 'CAT', 'CAT', 'DOG']) - result = words | count_implementation - assert_that( - result, equal_to([('CAT', (3 * factor)), ('DOG', (2 * factor))])) - p.run() + with TestPipeline() as p: + words = p | beam.Create(['CAT', 'DOG', 'CAT', 'CAT', 'DOG']) + result = words | count_implementation + assert_that( + result, equal_to([('CAT', (3 * factor)), ('DOG', (2 * factor))])) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py b/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py index 411feb8058a7..7161cff1c255 100644 --- a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py +++ b/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py @@ -135,18 +135,15 @@ def make_entity(self, content): def write_to_datastore(project, user_options, pipeline_options): """Creates a pipeline that writes entities to Cloud Datastore.""" - p = beam.Pipeline(options=pipeline_options) - - # pylint: disable=expression-not-assigned - (p - | 'read' >> ReadFromText(user_options.input) - | 'create entity' >> beam.Map( - EntityWrapper(user_options.namespace, user_options.kind, - user_options.ancestor).make_entity) - | 'write to datastore' >> WriteToDatastore(project)) + with beam.Pipeline(options=pipeline_options) as p: - # Actually run the pipeline (all operations above are deferred). - p.run().wait_until_finish() + # pylint: disable=expression-not-assigned + (p + | 'read' >> ReadFromText(user_options.input) + | 'create entity' >> beam.Map( + EntityWrapper(user_options.namespace, user_options.kind, + user_options.ancestor).make_entity) + | 'write to datastore' >> WriteToDatastore(project)) def make_ancestor_query(kind, namespace, ancestor): @@ -196,7 +193,6 @@ def read_from_datastore(project, user_options, pipeline_options): output | 'write' >> beam.io.WriteToText(file_path_prefix=user_options.output, num_shards=user_options.num_shards) - # Actually run the pipeline (all operations above are deferred). result = p.run() # Wait until completion, main thread would access post-completion job results. result.wait_until_finish() diff --git a/sdks/python/apache_beam/examples/cookbook/filters.py b/sdks/python/apache_beam/examples/cookbook/filters.py index 374001cd83fe..1fbf763e5005 100644 --- a/sdks/python/apache_beam/examples/cookbook/filters.py +++ b/sdks/python/apache_beam/examples/cookbook/filters.py @@ -86,20 +86,17 @@ def run(argv=None): help='Numeric value of month to filter on.') known_args, pipeline_args = parser.parse_known_args(argv) - p = beam.Pipeline(argv=pipeline_args) + with beam.Pipeline(argv=pipeline_args) as p: - input_data = p | beam.io.Read(beam.io.BigQuerySource(known_args.input)) + input_data = p | beam.io.Read(beam.io.BigQuerySource(known_args.input)) - # pylint: disable=expression-not-assigned - (filter_cold_days(input_data, known_args.month_filter) - | 'SaveToBQ' >> beam.io.Write(beam.io.BigQuerySink( - known_args.output, - schema='year:INTEGER,month:INTEGER,day:INTEGER,mean_temp:FLOAT', - create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, - write_disposition=beam.io.BigQueryDisposition.WRITE_TRUNCATE))) - - # Actually run the pipeline (all operations above are deferred). - p.run() + # pylint: disable=expression-not-assigned + (filter_cold_days(input_data, known_args.month_filter) + | 'SaveToBQ' >> beam.io.Write(beam.io.BigQuerySink( + known_args.output, + schema='year:INTEGER,month:INTEGER,day:INTEGER,mean_temp:FLOAT', + create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, + write_disposition=beam.io.BigQueryDisposition.WRITE_TRUNCATE))) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/group_with_coder.py b/sdks/python/apache_beam/examples/cookbook/group_with_coder.py index 6bdadae9f3d1..9c0d04b816ac 100644 --- a/sdks/python/apache_beam/examples/cookbook/group_with_coder.py +++ b/sdks/python/apache_beam/examples/cookbook/group_with_coder.py @@ -95,28 +95,27 @@ def run(args=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) - - # Register the custom coder for the Player class, so that it will be used in - # the computation. - coders.registry.register_coder(Player, PlayerCoder) - - (p # pylint: disable=expression-not-assigned - | ReadFromText(known_args.input) - # The get_players function is annotated with a type hint above, so the type - # system knows the output type of the following operation is a key-value pair - # of a Player and an int. Please see the documentation for details on - # types that are inferred automatically as well as other ways to specify - # type hints. - | beam.Map(get_players) - # The output type hint of the previous step is used to infer that the key - # type of the following operation is the Player type. Since a custom coder - # is registered for the Player class above, a PlayerCoder will be used to - # encode Player objects as keys for this combine operation. - | beam.CombinePerKey(sum) - | beam.Map(lambda (k, v): '%s,%d' % (k.name, v)) - | WriteToText(known_args.output)) - return p.run() + with beam.Pipeline(options=pipeline_options) as p: + + # Register the custom coder for the Player class, so that it will be used in + # the computation. + coders.registry.register_coder(Player, PlayerCoder) + + (p # pylint: disable=expression-not-assigned + | ReadFromText(known_args.input) + # The get_players function is annotated with a type hint above, so the type + # system knows the output type of the following operation is a key-value + # pair of a Player and an int. Please see the documentation for details on + # types that are inferred automatically as well as other ways to specify + # type hints. + | beam.Map(get_players) + # The output type hint of the previous step is used to infer that the key + # type of the following operation is the Player type. Since a custom coder + # is registered for the Player class above, a PlayerCoder will be used to + # encode Player objects as keys for this combine operation. + | beam.CombinePerKey(sum) + | beam.Map(lambda (k, v): '%s,%d' % (k.name, v)) + | WriteToText(known_args.output)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py b/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py index 4e8796647da0..268ba8d355f2 100644 --- a/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py +++ b/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py @@ -50,7 +50,7 @@ def test_basics_with_type_check(self): temp_path = self.create_temp_file(self.SAMPLE_RECORDS) group_with_coder.run([ '--input=%s*' % temp_path, - '--output=%s.result' % temp_path]).wait_until_finish() + '--output=%s.result' % temp_path]) # Parse result file and compare. results = [] with open(temp_path + '.result-00000-of-00001') as result_file: @@ -71,7 +71,7 @@ def test_basics_without_type_check(self): group_with_coder.run([ '--no_pipeline_type_check', '--input=%s*' % temp_path, - '--output=%s.result' % temp_path]).wait_until_finish() + '--output=%s.result' % temp_path]) # Parse result file and compare. results = [] with open(temp_path + '.result-00000-of-00001') as result_file: diff --git a/sdks/python/apache_beam/examples/cookbook/mergecontacts.py b/sdks/python/apache_beam/examples/cookbook/mergecontacts.py index 4f53c615ba97..9acdd9073478 100644 --- a/sdks/python/apache_beam/examples/cookbook/mergecontacts.py +++ b/sdks/python/apache_beam/examples/cookbook/mergecontacts.py @@ -70,64 +70,63 @@ def run(argv=None, assert_results=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) - - # Helper: read a tab-separated key-value mapping from a text file, escape all - # quotes/backslashes, and convert it a PCollection of (key, value) pairs. - def read_kv_textfile(label, textfile): - return (p - | 'Read: %s' % label >> ReadFromText(textfile) - | 'Backslash: %s' % label >> beam.Map( - lambda x: re.sub(r'\\', r'\\\\', x)) - | 'EscapeQuotes: %s' % label >> beam.Map( - lambda x: re.sub(r'"', r'\"', x)) - | 'Split: %s' % label >> beam.Map( - lambda x: re.split(r'\t+', x, 1))) - - # Read input databases. - email = read_kv_textfile('email', known_args.input_email) - phone = read_kv_textfile('phone', known_args.input_phone) - snailmail = read_kv_textfile('snailmail', known_args.input_snailmail) - - # Group together all entries under the same name. - grouped = (email, phone, snailmail) | 'group_by_name' >> beam.CoGroupByKey() - - # Prepare tab-delimited output; something like this: - # "name""email_1,email_2""phone""first_snailmail_only" - tsv_lines = grouped | beam.Map( - lambda (name, (email, phone, snailmail)): '\t'.join( - ['"%s"' % name, - '"%s"' % ','.join(email), - '"%s"' % ','.join(phone), - '"%s"' % next(iter(snailmail), '')])) - - # Compute some stats about our database of people. - luddites = grouped | beam.Filter( # People without email. - lambda (name, (email, phone, snailmail)): not next(iter(email), None)) - writers = grouped | beam.Filter( # People without phones. - lambda (name, (email, phone, snailmail)): not next(iter(phone), None)) - nomads = grouped | beam.Filter( # People without addresses. - lambda (name, (email, phone, snailmail)): not next(iter(snailmail), None)) - - num_luddites = luddites | 'Luddites' >> beam.combiners.Count.Globally() - num_writers = writers | 'Writers' >> beam.combiners.Count.Globally() - num_nomads = nomads | 'Nomads' >> beam.combiners.Count.Globally() - - # Write tab-delimited output. - # pylint: disable=expression-not-assigned - tsv_lines | 'WriteTsv' >> WriteToText(known_args.output_tsv) - - # TODO(silviuc): Move the assert_results logic to the unit test. - if assert_results is not None: - expected_luddites, expected_writers, expected_nomads = assert_results - assert_that(num_luddites, equal_to([expected_luddites]), - label='assert:luddites') - assert_that(num_writers, equal_to([expected_writers]), - label='assert:writers') - assert_that(num_nomads, equal_to([expected_nomads]), - label='assert:nomads') - # Execute pipeline. - return p.run() + with beam.Pipeline(options=pipeline_options) as p: + + # Helper: read a tab-separated key-value mapping from a text file, + # escape all quotes/backslashes, and convert it a PCollection of + # (key, value) pairs. + def read_kv_textfile(label, textfile): + return (p + | 'Read: %s' % label >> ReadFromText(textfile) + | 'Backslash: %s' % label >> beam.Map( + lambda x: re.sub(r'\\', r'\\\\', x)) + | 'EscapeQuotes: %s' % label >> beam.Map( + lambda x: re.sub(r'"', r'\"', x)) + | 'Split: %s' % label >> beam.Map( + lambda x: re.split(r'\t+', x, 1))) + + # Read input databases. + email = read_kv_textfile('email', known_args.input_email) + phone = read_kv_textfile('phone', known_args.input_phone) + snailmail = read_kv_textfile('snailmail', known_args.input_snailmail) + + # Group together all entries under the same name. + grouped = (email, phone, snailmail) | 'group_by_name' >> beam.CoGroupByKey() + + # Prepare tab-delimited output; something like this: + # "name""email_1,email_2""phone""first_snailmail_only" + tsv_lines = grouped | beam.Map( + lambda (name, (email, phone, snailmail)): '\t'.join( + ['"%s"' % name, + '"%s"' % ','.join(email), + '"%s"' % ','.join(phone), + '"%s"' % next(iter(snailmail), '')])) + + # Compute some stats about our database of people. + luddites = grouped | beam.Filter( # People without email. + lambda (name, (email, phone, snailmail)): not next(iter(email), None)) + writers = grouped | beam.Filter( # People without phones. + lambda (name, (email, phone, snailmail)): not next(iter(phone), None)) + nomads = grouped | beam.Filter( # People without addresses. + lambda (name, (e, p, snailmail)): not next(iter(snailmail), None)) + + num_luddites = luddites | 'Luddites' >> beam.combiners.Count.Globally() + num_writers = writers | 'Writers' >> beam.combiners.Count.Globally() + num_nomads = nomads | 'Nomads' >> beam.combiners.Count.Globally() + + # Write tab-delimited output. + # pylint: disable=expression-not-assigned + tsv_lines | 'WriteTsv' >> WriteToText(known_args.output_tsv) + + # TODO(silviuc): Move the assert_results logic to the unit test. + if assert_results is not None: + expected_luddites, expected_writers, expected_nomads = assert_results + assert_that(num_luddites, equal_to([expected_luddites]), + label='assert:luddites') + assert_that(num_writers, equal_to([expected_writers]), + label='assert:writers') + assert_that(num_nomads, equal_to([expected_nomads]), + label='assert:nomads') if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/mergecontacts_test.py b/sdks/python/apache_beam/examples/cookbook/mergecontacts_test.py index 09f71d389696..b3be0ddb0853 100644 --- a/sdks/python/apache_beam/examples/cookbook/mergecontacts_test.py +++ b/sdks/python/apache_beam/examples/cookbook/mergecontacts_test.py @@ -107,13 +107,12 @@ def test_mergecontacts(self): result_prefix = self.create_temp_file('') - result = mergecontacts.run([ + mergecontacts.run([ '--input_email=%s' % path_email, '--input_phone=%s' % path_phone, '--input_snailmail=%s' % path_snailmail, '--output_tsv=%s.tsv' % result_prefix, '--output_stats=%s.stats' % result_prefix], assert_results=(2, 1, 3)) - result.wait_until_finish() with open('%s.tsv-00000-of-00001' % result_prefix) as f: contents = f.read() diff --git a/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo.py b/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo.py index 9759f4821045..2316c6611c06 100644 --- a/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo.py +++ b/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo.py @@ -141,43 +141,41 @@ def run(argv=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) - - lines = p | ReadFromText(known_args.input) - - # with_outputs allows accessing the explicitly tagged outputs of a DoFn. - split_lines_result = (lines - | beam.ParDo(SplitLinesToWordsFn()).with_outputs( - SplitLinesToWordsFn.OUTPUT_TAG_SHORT_WORDS, - SplitLinesToWordsFn.OUTPUT_TAG_CHARACTER_COUNT, - main='words')) - - # split_lines_result is an object of type DoOutputsTuple. It supports - # accessing result in alternative ways. - words, _, _ = split_lines_result - short_words = split_lines_result[ - SplitLinesToWordsFn.OUTPUT_TAG_SHORT_WORDS] - character_count = split_lines_result.tag_character_count - - # pylint: disable=expression-not-assigned - (character_count - | 'pair_with_key' >> beam.Map(lambda x: ('chars_temp_key', x)) - | beam.GroupByKey() - | 'count chars' >> beam.Map(lambda (_, counts): sum(counts)) - | 'write chars' >> WriteToText(known_args.output + '-chars')) - - # pylint: disable=expression-not-assigned - (short_words - | 'count short words' >> CountWords() - | 'write short words' >> WriteToText( - known_args.output + '-short-words')) - - # pylint: disable=expression-not-assigned - (words - | 'count words' >> CountWords() - | 'write words' >> WriteToText(known_args.output + '-words')) - - return p.run() + with beam.Pipeline(options=pipeline_options) as p: + + lines = p | ReadFromText(known_args.input) + + # with_outputs allows accessing the explicitly tagged outputs of a DoFn. + split_lines_result = (lines + | beam.ParDo(SplitLinesToWordsFn()).with_outputs( + SplitLinesToWordsFn.OUTPUT_TAG_SHORT_WORDS, + SplitLinesToWordsFn.OUTPUT_TAG_CHARACTER_COUNT, + main='words')) + + # split_lines_result is an object of type DoOutputsTuple. It supports + # accessing result in alternative ways. + words, _, _ = split_lines_result + short_words = split_lines_result[ + SplitLinesToWordsFn.OUTPUT_TAG_SHORT_WORDS] + character_count = split_lines_result.tag_character_count + + # pylint: disable=expression-not-assigned + (character_count + | 'pair_with_key' >> beam.Map(lambda x: ('chars_temp_key', x)) + | beam.GroupByKey() + | 'count chars' >> beam.Map(lambda (_, counts): sum(counts)) + | 'write chars' >> WriteToText(known_args.output + '-chars')) + + # pylint: disable=expression-not-assigned + (short_words + | 'count short words' >> CountWords() + | 'write short words' >> WriteToText( + known_args.output + '-short-words')) + + # pylint: disable=expression-not-assigned + (words + | 'count words' >> CountWords() + | 'write words' >> WriteToText(known_args.output + '-words')) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo_test.py b/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo_test.py index 2c9111c678bd..3ddd668599d0 100644 --- a/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo_test.py +++ b/sdks/python/apache_beam/examples/cookbook/multiple_output_pardo_test.py @@ -52,7 +52,7 @@ def test_multiple_output_pardo(self): multiple_output_pardo.run([ '--input=%s*' % temp_path, - '--output=%s' % result_prefix]).wait_until_finish() + '--output=%s' % result_prefix]) expected_char_count = len(''.join(self.SAMPLE_TEXT.split('\n'))) with open(result_prefix + '-chars-00000-of-00001') as f: diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py index 72595723de45..70929e9e8fae 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets.py +++ b/sdks/python/apache_beam/examples/snippets/snippets.py @@ -147,18 +147,15 @@ def _add_argparse_args(cls, parser): pipeline_options = PipelineOptions(argv) my_options = pipeline_options.view_as(MyOptions) - p = beam.Pipeline(options=pipeline_options) - - (p - | beam.io.ReadFromText(my_options.input) - | beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) - | beam.Map(lambda x: (x, 1)) - | beam.combiners.Count.PerKey() - | beam.io.WriteToText(my_options.output)) - - result = p.run() + with beam.Pipeline(options=pipeline_options) as p: + + (p + | beam.io.ReadFromText(my_options.input) + | beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) + | beam.Map(lambda x: (x, 1)) + | beam.combiners.Count.PerKey() + | beam.io.WriteToText(my_options.output)) # [END model_pipelines] - result.wait_until_finish() def model_pcollection(argv): @@ -178,21 +175,18 @@ def _add_argparse_args(cls, parser): my_options = pipeline_options.view_as(MyOptions) # [START model_pcollection] - p = beam.Pipeline(options=pipeline_options) + with beam.Pipeline(options=pipeline_options) as p: - lines = (p - | beam.Create([ - 'To be, or not to be: that is the question: ', - 'Whether \'tis nobler in the mind to suffer ', - 'The slings and arrows of outrageous fortune, ', - 'Or to take arms against a sea of troubles, '])) - # [END model_pcollection] + lines = (p + | beam.Create([ + 'To be, or not to be: that is the question: ', + 'Whether \'tis nobler in the mind to suffer ', + 'The slings and arrows of outrageous fortune, ', + 'Or to take arms against a sea of troubles, '])) + # [END model_pcollection] - (lines - | beam.io.WriteToText(my_options.output)) - - result = p.run() - result.wait_until_finish() + (lines + | beam.io.WriteToText(my_options.output)) def pipeline_options_remote(argv): @@ -297,12 +291,10 @@ def pipeline_options_command_line(argv): known_args, pipeline_args = parser.parse_known_args(argv) # Create the Pipeline with remaining arguments. - p = beam.Pipeline(argv=pipeline_args) - lines = p | 'ReadFromText' >> beam.io.ReadFromText(known_args.input) - lines | 'WriteToText' >> beam.io.WriteToText(known_args.output) - # [END pipeline_options_command_line] - - p.run().wait_until_finish() + with beam.Pipeline(argv=pipeline_args) as p: + lines = p | 'ReadFromText' >> beam.io.ReadFromText(known_args.input) + lines | 'WriteToText' >> beam.io.WriteToText(known_args.output) + # [END pipeline_options_command_line] def pipeline_logging(lines, output): @@ -329,13 +321,11 @@ def process(self, element): # Remaining WordCount example code ... # [END pipeline_logging] - p = TestPipeline() # Use TestPipeline for testing. - (p - | beam.Create(lines) - | beam.ParDo(ExtractWordsFn()) - | beam.io.WriteToText(output)) - - p.run() + with TestPipeline() as p: # Use TestPipeline for testing. + (p + | beam.Create(lines) + | beam.ParDo(ExtractWordsFn()) + | beam.io.WriteToText(output)) def pipeline_monitoring(renames): @@ -385,20 +375,19 @@ def expand(self, pcoll): pipeline_options = PipelineOptions() options = pipeline_options.view_as(WordCountOptions) - p = TestPipeline() # Use TestPipeline for testing. + with TestPipeline() as p: # Use TestPipeline for testing. - # [START pipeline_monitoring_execution] - (p - # Read the lines of the input text. - | 'ReadLines' >> beam.io.ReadFromText(options.input) - # Count the words. - | CountWords() - # Write the formatted word counts to output. - | 'WriteCounts' >> beam.io.WriteToText(options.output)) - # [END pipeline_monitoring_execution] + # [START pipeline_monitoring_execution] + (p + # Read the lines of the input text. + | 'ReadLines' >> beam.io.ReadFromText(options.input) + # Count the words. + | CountWords() + # Write the formatted word counts to output. + | 'WriteCounts' >> beam.io.WriteToText(options.output)) + # [END pipeline_monitoring_execution] - p.visit(SnippetUtils.RenameFiles(renames)) - p.run() + p.visit(SnippetUtils.RenameFiles(renames)) def examples_wordcount_minimal(renames): @@ -478,40 +467,39 @@ def _add_argparse_args(cls, parser): default='gs://my-bucket/input') options = PipelineOptions(argv) - p = beam.Pipeline(options=options) - # [END examples_wordcount_wordcount_options] + with beam.Pipeline(options=options) as p: + # [END examples_wordcount_wordcount_options] - lines = p | beam.io.ReadFromText( - 'gs://dataflow-samples/shakespeare/kinglear.txt') + lines = p | beam.io.ReadFromText( + 'gs://dataflow-samples/shakespeare/kinglear.txt') - # [START examples_wordcount_wordcount_composite] - class CountWords(beam.PTransform): + # [START examples_wordcount_wordcount_composite] + class CountWords(beam.PTransform): - def expand(self, pcoll): - return (pcoll - # Convert lines of text into individual words. - | 'ExtractWords' >> beam.FlatMap( - lambda x: re.findall(r'[A-Za-z\']+', x)) + def expand(self, pcoll): + return (pcoll + # Convert lines of text into individual words. + | 'ExtractWords' >> beam.FlatMap( + lambda x: re.findall(r'[A-Za-z\']+', x)) - # Count the number of times each word occurs. - | beam.combiners.Count.PerElement()) + # Count the number of times each word occurs. + | beam.combiners.Count.PerElement()) - counts = lines | CountWords() - # [END examples_wordcount_wordcount_composite] + counts = lines | CountWords() + # [END examples_wordcount_wordcount_composite] - # [START examples_wordcount_wordcount_dofn] - class FormatAsTextFn(beam.DoFn): + # [START examples_wordcount_wordcount_dofn] + class FormatAsTextFn(beam.DoFn): - def process(self, element): - word, count = element - yield '%s: %s' % (word, count) + def process(self, element): + word, count = element + yield '%s: %s' % (word, count) - formatted = counts | beam.ParDo(FormatAsTextFn()) - # [END examples_wordcount_wordcount_dofn] + formatted = counts | beam.ParDo(FormatAsTextFn()) + # [END examples_wordcount_wordcount_dofn] - formatted | beam.io.WriteToText('gs://my-bucket/counts.txt') - p.visit(SnippetUtils.RenameFiles(renames)) - p.run().wait_until_finish() + formatted | beam.io.WriteToText('gs://my-bucket/counts.txt') + p.visit(SnippetUtils.RenameFiles(renames)) def examples_wordcount_debugging(renames): @@ -558,27 +546,27 @@ def process(self, element): # [END example_wordcount_debugging_logging] # [END example_wordcount_debugging_aggregators] - p = TestPipeline() # Use TestPipeline for testing. - filtered_words = ( - p - | beam.io.ReadFromText( - 'gs://dataflow-samples/shakespeare/kinglear.txt') - | 'ExtractWords' >> beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) - | beam.combiners.Count.PerElement() - | 'FilterText' >> beam.ParDo(FilterTextFn('Flourish|stomach'))) + with TestPipeline() as p: # Use TestPipeline for testing. + filtered_words = ( + p + | beam.io.ReadFromText( + 'gs://dataflow-samples/shakespeare/kinglear.txt') + | 'ExtractWords' >> beam.FlatMap( + lambda x: re.findall(r'[A-Za-z\']+', x)) + | beam.combiners.Count.PerElement() + | 'FilterText' >> beam.ParDo(FilterTextFn('Flourish|stomach'))) - # [START example_wordcount_debugging_assert] - beam.testing.util.assert_that( - filtered_words, beam.testing.util.equal_to( - [('Flourish', 3), ('stomach', 1)])) - # [END example_wordcount_debugging_assert] + # [START example_wordcount_debugging_assert] + beam.testing.util.assert_that( + filtered_words, beam.testing.util.equal_to( + [('Flourish', 3), ('stomach', 1)])) + # [END example_wordcount_debugging_assert] - output = (filtered_words - | 'format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) - | 'Write' >> beam.io.WriteToText('gs://my-bucket/counts.txt')) + output = (filtered_words + | 'format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) + | 'Write' >> beam.io.WriteToText('gs://my-bucket/counts.txt')) - p.visit(SnippetUtils.RenameFiles(renames)) - p.run() + p.visit(SnippetUtils.RenameFiles(renames)) import apache_beam as beam @@ -659,16 +647,14 @@ def model_custom_source(count): # Using the source in an example pipeline. # [START model_custom_source_use_new_source] - p = beam.Pipeline(options=PipelineOptions()) - numbers = p | 'ProduceNumbers' >> beam.io.Read(CountingSource(count)) - # [END model_custom_source_use_new_source] + with beam.Pipeline(options=PipelineOptions()) as p: + numbers = p | 'ProduceNumbers' >> beam.io.Read(CountingSource(count)) + # [END model_custom_source_use_new_source] - lines = numbers | beam.core.Map(lambda number: 'line %d' % number) - assert_that( - lines, equal_to( - ['line ' + str(number) for number in range(0, count)])) - - p.run().wait_until_finish() + lines = numbers | beam.core.Map(lambda number: 'line %d' % number) + assert_that( + lines, equal_to( + ['line ' + str(number) for number in range(0, count)])) # We recommend users to start Source classes with an underscore to discourage # using the Source class directly when a PTransform for the source is @@ -796,14 +782,12 @@ def close(self): # Using the new sink in an example pipeline. # [START model_custom_sink_use_new_sink] - p = beam.Pipeline(options=PipelineOptions()) - kvs = p | 'CreateKVs' >> beam.Create(KVs) + with beam.Pipeline(options=PipelineOptions()) as p: + kvs = p | 'CreateKVs' >> beam.Create(KVs) - kvs | 'WriteToSimpleKV' >> beam.io.Write( - SimpleKVSink('http://url_to_simple_kv/', final_table_name)) - # [END model_custom_sink_use_new_sink] - - p.run().wait_until_finish() + kvs | 'WriteToSimpleKV' >> beam.io.Write( + SimpleKVSink('http://url_to_simple_kv/', final_table_name)) + # [END model_custom_sink_use_new_sink] # We recommend users to start Sink class names with an underscore to # discourage using the Sink class directly when a PTransform for the sink is @@ -828,13 +812,11 @@ def expand(self, pcoll): final_table_name = final_table_name_with_ptransform # [START model_custom_sink_use_ptransform] - p = beam.Pipeline(options=PipelineOptions()) - kvs = p | 'CreateKVs' >> beam.core.Create(KVs) - kvs | 'WriteToSimpleKV' >> WriteToKVSink( - 'http://url_to_simple_kv/', final_table_name) - # [END model_custom_sink_use_ptransform] - - p.run().wait_until_finish() + with beam.Pipeline(options=PipelineOptions()) as p: + kvs = p | 'CreateKVs' >> beam.core.Create(KVs) + kvs | 'WriteToSimpleKV' >> WriteToKVSink( + 'http://url_to_simple_kv/', final_table_name) + # [END model_custom_sink_use_ptransform] def model_textio(renames): @@ -847,37 +829,35 @@ def filter_words(x): from apache_beam.options.pipeline_options import PipelineOptions # [START model_textio_read] - p = beam.Pipeline(options=PipelineOptions()) - # [START model_pipelineio_read] - lines = p | 'ReadFromText' >> beam.io.ReadFromText('path/to/input-*.csv') - # [END model_pipelineio_read] - # [END model_textio_read] - - # [START model_textio_write] - filtered_words = lines | 'FilterWords' >> beam.FlatMap(filter_words) - # [START model_pipelineio_write] - filtered_words | 'WriteToText' >> beam.io.WriteToText( - '/path/to/numbers', file_name_suffix='.csv') - # [END model_pipelineio_write] - # [END model_textio_write] + with beam.Pipeline(options=PipelineOptions()) as p: + # [START model_pipelineio_read] + lines = p | 'ReadFromText' >> beam.io.ReadFromText('path/to/input-*.csv') + # [END model_pipelineio_read] + # [END model_textio_read] - p.visit(SnippetUtils.RenameFiles(renames)) - p.run().wait_until_finish() + # [START model_textio_write] + filtered_words = lines | 'FilterWords' >> beam.FlatMap(filter_words) + # [START model_pipelineio_write] + filtered_words | 'WriteToText' >> beam.io.WriteToText( + '/path/to/numbers', file_name_suffix='.csv') + # [END model_pipelineio_write] + # [END model_textio_write] + + p.visit(SnippetUtils.RenameFiles(renames)) def model_textio_compressed(renames, expected): """Using a Read Transform to read compressed text files.""" - p = TestPipeline() + with TestPipeline() as p: - # [START model_textio_write_compressed] - lines = p | 'ReadFromText' >> beam.io.ReadFromText( - '/path/to/input-*.csv.gz', - compression_type=beam.io.filesystem.CompressionTypes.GZIP) - # [END model_textio_write_compressed] + # [START model_textio_write_compressed] + lines = p | 'ReadFromText' >> beam.io.ReadFromText( + '/path/to/input-*.csv.gz', + compression_type=beam.io.filesystem.CompressionTypes.GZIP) + # [END model_textio_write_compressed] - assert_that(lines, equal_to(expected)) - p.visit(SnippetUtils.RenameFiles(renames)) - p.run().wait_until_finish() + assert_that(lines, equal_to(expected)) + p.visit(SnippetUtils.RenameFiles(renames)) def model_datastoreio(): @@ -987,43 +967,40 @@ def expand(self, pcoll): # [END composite_ptransform_apply_method] # [END composite_transform_example] - p = TestPipeline() # Use TestPipeline for testing. - (p - | beam.Create(contents) - | CountWords() - | beam.io.WriteToText(output_path)) - p.run() + with TestPipeline() as p: # Use TestPipeline for testing. + (p + | beam.Create(contents) + | CountWords() + | beam.io.WriteToText(output_path)) def model_multiple_pcollections_flatten(contents, output_path): """Merging a PCollection with Flatten.""" some_hash_fn = lambda s: ord(s[0]) - import apache_beam as beam - p = TestPipeline() # Use TestPipeline for testing. partition_fn = lambda element, partitions: some_hash_fn(element) % partitions - - # Partition into deciles - partitioned = p | beam.Create(contents) | beam.Partition(partition_fn, 3) - pcoll1 = partitioned[0] - pcoll2 = partitioned[1] - pcoll3 = partitioned[2] - - # Flatten them back into 1 - - # A collection of PCollection objects can be represented simply - # as a tuple (or list) of PCollections. - # (The SDK for Python has no separate type to store multiple - # PCollection objects, whether containing the same or different - # types.) - # [START model_multiple_pcollections_flatten] - merged = ( - (pcoll1, pcoll2, pcoll3) - # A list of tuples can be "piped" directly into a Flatten transform. - | beam.Flatten()) - # [END model_multiple_pcollections_flatten] - merged | beam.io.WriteToText(output_path) - - p.run() + import apache_beam as beam + with TestPipeline() as p: # Use TestPipeline for testing. + + # Partition into deciles + partitioned = p | beam.Create(contents) | beam.Partition(partition_fn, 3) + pcoll1 = partitioned[0] + pcoll2 = partitioned[1] + pcoll3 = partitioned[2] + + # Flatten them back into 1 + + # A collection of PCollection objects can be represented simply + # as a tuple (or list) of PCollections. + # (The SDK for Python has no separate type to store multiple + # PCollection objects, whether containing the same or different + # types.) + # [START model_multiple_pcollections_flatten] + merged = ( + (pcoll1, pcoll2, pcoll3) + # A list of tuples can be "piped" directly into a Flatten transform. + | beam.Flatten()) + # [END model_multiple_pcollections_flatten] + merged | beam.io.WriteToText(output_path) def model_multiple_pcollections_partition(contents, output_path): @@ -1034,25 +1011,23 @@ def get_percentile(i): """Assume i in [0,100).""" return i import apache_beam as beam - p = TestPipeline() # Use TestPipeline for testing. + with TestPipeline() as p: # Use TestPipeline for testing. - students = p | beam.Create(contents) + students = p | beam.Create(contents) - # [START model_multiple_pcollections_partition] - def partition_fn(student, num_partitions): - return int(get_percentile(student) * num_partitions / 100) + # [START model_multiple_pcollections_partition] + def partition_fn(student, num_partitions): + return int(get_percentile(student) * num_partitions / 100) - by_decile = students | beam.Partition(partition_fn, 10) - # [END model_multiple_pcollections_partition] - # [START model_multiple_pcollections_partition_40th] - fortieth_percentile = by_decile[4] - # [END model_multiple_pcollections_partition_40th] + by_decile = students | beam.Partition(partition_fn, 10) + # [END model_multiple_pcollections_partition] + # [START model_multiple_pcollections_partition_40th] + fortieth_percentile = by_decile[4] + # [END model_multiple_pcollections_partition_40th] - ([by_decile[d] for d in xrange(10) if d != 4] + [fortieth_percentile] - | beam.Flatten() - | beam.io.WriteToText(output_path)) - - p.run() + ([by_decile[d] for d in xrange(10) if d != 4] + [fortieth_percentile] + | beam.Flatten() + | beam.io.WriteToText(output_path)) def model_group_by_key(contents, output_path): @@ -1060,58 +1035,56 @@ def model_group_by_key(contents, output_path): import re import apache_beam as beam - p = TestPipeline() # Use TestPipeline for testing. - words_and_counts = ( - p - | beam.Create(contents) - | beam.FlatMap(lambda x: re.findall(r'\w+', x)) - | 'one word' >> beam.Map(lambda w: (w, 1))) - # GroupByKey accepts a PCollection of (w, 1) and - # outputs a PCollection of (w, (1, 1, ...)). - # (A key/value pair is just a tuple in Python.) - # This is a somewhat forced example, since one could - # simply use beam.combiners.Count.PerElement here. - # [START model_group_by_key_transform] - grouped_words = words_and_counts | beam.GroupByKey() - # [END model_group_by_key_transform] - (grouped_words - | 'count words' >> beam.Map(lambda (word, counts): (word, len(counts))) - | beam.io.WriteToText(output_path)) - p.run() + with TestPipeline() as p: # Use TestPipeline for testing. + words_and_counts = ( + p + | beam.Create(contents) + | beam.FlatMap(lambda x: re.findall(r'\w+', x)) + | 'one word' >> beam.Map(lambda w: (w, 1))) + # GroupByKey accepts a PCollection of (w, 1) and + # outputs a PCollection of (w, (1, 1, ...)). + # (A key/value pair is just a tuple in Python.) + # This is a somewhat forced example, since one could + # simply use beam.combiners.Count.PerElement here. + # [START model_group_by_key_transform] + grouped_words = words_and_counts | beam.GroupByKey() + # [END model_group_by_key_transform] + (grouped_words + | 'count words' >> beam.Map(lambda (word, counts): (word, len(counts))) + | beam.io.WriteToText(output_path)) def model_co_group_by_key_tuple(email_list, phone_list, output_path): """Applying a CoGroupByKey Transform to a tuple.""" import apache_beam as beam - p = TestPipeline() # Use TestPipeline for testing. - # [START model_group_by_key_cogroupbykey_tuple] - # Each data set is represented by key-value pairs in separate PCollections. - # Both data sets share a common key type (in this example str). - # The email_list contains values such as: ('joe', 'joe@example.com') with - # multiple possible values for each key. - # The phone_list contains values such as: ('mary': '111-222-3333') with - # multiple possible values for each key. - emails = p | 'email' >> beam.Create(email_list) - phones = p | 'phone' >> beam.Create(phone_list) - # The result PCollection contains one key-value element for each key in the - # input PCollections. The key of the pair will be the key from the input and - # the value will be a dictionary with two entries: 'emails' - an iterable of - # all values for the current key in the emails PCollection and 'phones': an - # iterable of all values for the current key in the phones PCollection. - # For instance, if 'emails' contained ('joe', 'joe@example.com') and - # ('joe', 'joe@gmail.com'), then 'result' will contain the element - # ('joe', {'emails': ['joe@example.com', 'joe@gmail.com'], 'phones': ...}) - result = {'emails': emails, 'phones': phones} | beam.CoGroupByKey() - - def join_info((name, info)): - return '; '.join(['%s' % name, - '%s' % ','.join(info['emails']), - '%s' % ','.join(info['phones'])]) - - contact_lines = result | beam.Map(join_info) - # [END model_group_by_key_cogroupbykey_tuple] - contact_lines | beam.io.WriteToText(output_path) - p.run() + with TestPipeline() as p: # Use TestPipeline for testing. + # [START model_group_by_key_cogroupbykey_tuple] + # Each data set is represented by key-value pairs in separate PCollections. + # Both data sets share a common key type (in this example str). + # The email_list contains values such as: ('joe', 'joe@example.com') with + # multiple possible values for each key. + # The phone_list contains values such as: ('mary': '111-222-3333') with + # multiple possible values for each key. + emails = p | 'email' >> beam.Create(email_list) + phones = p | 'phone' >> beam.Create(phone_list) + # The result PCollection contains one key-value element for each key in the + # input PCollections. The key of the pair will be the key from the input and + # the value will be a dictionary with two entries: 'emails' - an iterable of + # all values for the current key in the emails PCollection and 'phones': an + # iterable of all values for the current key in the phones PCollection. + # For instance, if 'emails' contained ('joe', 'joe@example.com') and + # ('joe', 'joe@gmail.com'), then 'result' will contain the element + # ('joe', {'emails': ['joe@example.com', 'joe@gmail.com'], 'phones': ...}) + result = {'emails': emails, 'phones': phones} | beam.CoGroupByKey() + + def join_info((name, info)): + return '; '.join(['%s' % name, + '%s' % ','.join(info['emails']), + '%s' % ','.join(info['phones'])]) + + contact_lines = result | beam.Map(join_info) + # [END model_group_by_key_cogroupbykey_tuple] + contact_lines | beam.io.WriteToText(output_path) def model_join_using_side_inputs( @@ -1121,35 +1094,34 @@ def model_join_using_side_inputs( import apache_beam as beam from apache_beam.pvalue import AsIter - p = TestPipeline() # Use TestPipeline for testing. - # [START model_join_using_side_inputs] - # This code performs a join by receiving the set of names as an input and - # passing PCollections that contain emails and phone numbers as side inputs - # instead of using CoGroupByKey. - names = p | 'names' >> beam.Create(name_list) - emails = p | 'email' >> beam.Create(email_list) - phones = p | 'phone' >> beam.Create(phone_list) - - def join_info(name, emails, phone_numbers): - filtered_emails = [] - for name_in_list, email in emails: - if name_in_list == name: - filtered_emails.append(email) - - filtered_phone_numbers = [] - for name_in_list, phone_number in phone_numbers: - if name_in_list == name: - filtered_phone_numbers.append(phone_number) - - return '; '.join(['%s' % name, - '%s' % ','.join(filtered_emails), - '%s' % ','.join(filtered_phone_numbers)]) - - contact_lines = names | 'CreateContacts' >> beam.core.Map( - join_info, AsIter(emails), AsIter(phones)) - # [END model_join_using_side_inputs] - contact_lines | beam.io.WriteToText(output_path) - p.run() + with TestPipeline() as p: # Use TestPipeline for testing. + # [START model_join_using_side_inputs] + # This code performs a join by receiving the set of names as an input and + # passing PCollections that contain emails and phone numbers as side inputs + # instead of using CoGroupByKey. + names = p | 'names' >> beam.Create(name_list) + emails = p | 'email' >> beam.Create(email_list) + phones = p | 'phone' >> beam.Create(phone_list) + + def join_info(name, emails, phone_numbers): + filtered_emails = [] + for name_in_list, email in emails: + if name_in_list == name: + filtered_emails.append(email) + + filtered_phone_numbers = [] + for name_in_list, phone_number in phone_numbers: + if name_in_list == name: + filtered_phone_numbers.append(phone_number) + + return '; '.join(['%s' % name, + '%s' % ','.join(filtered_emails), + '%s' % ','.join(filtered_phone_numbers)]) + + contact_lines = names | 'CreateContacts' >> beam.core.Map( + join_info, AsIter(emails), AsIter(phones)) + # [END model_join_using_side_inputs] + contact_lines | beam.io.WriteToText(output_path) # [START model_library_transforms_keys] diff --git a/sdks/python/apache_beam/examples/snippets/snippets_test.py b/sdks/python/apache_beam/examples/snippets/snippets_test.py index f7b51a75e7eb..6654fef654cb 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets_test.py +++ b/sdks/python/apache_beam/examples/snippets/snippets_test.py @@ -119,7 +119,6 @@ def capitals(word): self.assertEqual({'A', 'C'}, set(all_capitals)) def test_pardo_with_label(self): - # pylint: disable=line-too-long words = ['aa', 'bbc', 'defg'] # [START model_pardo_with_label] result = words | 'CountUniqueLetters' >> beam.Map( @@ -129,41 +128,41 @@ def test_pardo_with_label(self): self.assertEqual({1, 2, 4}, set(result)) def test_pardo_side_input(self): - p = TestPipeline() - words = p | 'start' >> beam.Create(['a', 'bb', 'ccc', 'dddd']) - - # [START model_pardo_side_input] - # Callable takes additional arguments. - def filter_using_length(word, lower_bound, upper_bound=float('inf')): - if lower_bound <= len(word) <= upper_bound: - yield word - - # Construct a deferred side input. - avg_word_len = (words - | beam.Map(len) - | beam.CombineGlobally(beam.combiners.MeanCombineFn())) - - # Call with explicit side inputs. - small_words = words | 'small' >> beam.FlatMap(filter_using_length, 0, 3) - - # A single deferred side input. - larger_than_average = (words | 'large' >> beam.FlatMap( - filter_using_length, - lower_bound=pvalue.AsSingleton(avg_word_len))) - - # Mix and match. - small_but_nontrivial = words | beam.FlatMap(filter_using_length, - lower_bound=2, - upper_bound=pvalue.AsSingleton( - avg_word_len)) - # [END model_pardo_side_input] - - assert_that(small_words, equal_to(['a', 'bb', 'ccc'])) - assert_that(larger_than_average, equal_to(['ccc', 'dddd']), - label='larger_than_average') - assert_that(small_but_nontrivial, equal_to(['bb']), - label='small_but_not_trivial') - p.run() + # pylint: disable=line-too-long + with TestPipeline() as p: + words = p | 'start' >> beam.Create(['a', 'bb', 'ccc', 'dddd']) + + # [START model_pardo_side_input] + # Callable takes additional arguments. + def filter_using_length(word, lower_bound, upper_bound=float('inf')): + if lower_bound <= len(word) <= upper_bound: + yield word + + # Construct a deferred side input. + avg_word_len = (words + | beam.Map(len) + | beam.CombineGlobally(beam.combiners.MeanCombineFn())) + + # Call with explicit side inputs. + small_words = words | 'small' >> beam.FlatMap(filter_using_length, 0, 3) + + # A single deferred side input. + larger_than_average = (words | 'large' >> beam.FlatMap( + filter_using_length, + lower_bound=pvalue.AsSingleton(avg_word_len))) + + # Mix and match. + small_but_nontrivial = words | beam.FlatMap( + filter_using_length, + lower_bound=2, + upper_bound=pvalue.AsSingleton(avg_word_len)) + # [END model_pardo_side_input] + + assert_that(small_words, equal_to(['a', 'bb', 'ccc'])) + assert_that(larger_than_average, equal_to(['ccc', 'dddd']), + label='larger_than_average') + assert_that(small_but_nontrivial, equal_to(['bb']), + label='small_but_not_trivial') def test_pardo_side_input_dofn(self): words = ['a', 'bb', 'ccc', 'dddd'] @@ -307,10 +306,9 @@ def expand(self, pcoll): def test_runtime_checks_off(self): # pylint: disable=expression-not-assigned - p = TestPipeline() - # [START type_hints_runtime_off] - p | beam.Create(['a']) | beam.Map(lambda x: 3).with_output_types(str) - p.run() + with TestPipeline() as p: + # [START type_hints_runtime_off] + p | beam.Create(['a']) | beam.Map(lambda x: 3).with_output_types(str) # [END type_hints_runtime_off] def test_runtime_checks_on(self): @@ -323,47 +321,45 @@ def test_runtime_checks_on(self): # [END type_hints_runtime_on] def test_deterministic_key(self): - p = TestPipeline() - lines = (p | beam.Create( - ['banana,fruit,3', 'kiwi,fruit,2', 'kiwi,fruit,2', 'zucchini,veg,3'])) + with TestPipeline() as p: + lines = (p | beam.Create( + ['banana,fruit,3', 'kiwi,fruit,2', 'kiwi,fruit,2', 'zucchini,veg,3'])) - # For pickling - global Player # pylint: disable=global-variable-not-assigned + # For pickling + global Player # pylint: disable=global-variable-not-assigned - # [START type_hints_deterministic_key] - class Player(object): - def __init__(self, team, name): - self.team = team - self.name = name + # [START type_hints_deterministic_key] + class Player(object): + def __init__(self, team, name): + self.team = team + self.name = name - class PlayerCoder(beam.coders.Coder): - def encode(self, player): - return '%s:%s' % (player.team, player.name) + class PlayerCoder(beam.coders.Coder): + def encode(self, player): + return '%s:%s' % (player.team, player.name) - def decode(self, s): - return Player(*s.split(':')) + def decode(self, s): + return Player(*s.split(':')) - def is_deterministic(self): - return True + def is_deterministic(self): + return True - beam.coders.registry.register_coder(Player, PlayerCoder) + beam.coders.registry.register_coder(Player, PlayerCoder) - def parse_player_and_score(csv): - name, team, score = csv.split(',') - return Player(team, name), int(score) + def parse_player_and_score(csv): + name, team, score = csv.split(',') + return Player(team, name), int(score) - totals = ( - lines - | beam.Map(parse_player_and_score) - | beam.CombinePerKey(sum).with_input_types( - beam.typehints.Tuple[Player, int])) - # [END type_hints_deterministic_key] + totals = ( + lines + | beam.Map(parse_player_and_score) + | beam.CombinePerKey(sum).with_input_types( + beam.typehints.Tuple[Player, int])) + # [END type_hints_deterministic_key] - assert_that( - totals | beam.Map(lambda (k, v): (k.name, v)), - equal_to([('banana', 3), ('kiwi', 4), ('zucchini', 3)])) - - p.run() + assert_that( + totals | beam.Map(lambda (k, v): (k.name, v)), + equal_to([('banana', 3), ('kiwi', 4), ('zucchini', 3)])) class SnippetsTest(unittest.TestCase): @@ -802,109 +798,104 @@ def test_count(self): self.assertEqual({('cat', 3), ('dog', 2)}, set(perkey_counts)) def test_setting_fixed_windows(self): - p = TestPipeline() - unkeyed_items = p | beam.Create([22, 33, 55, 100, 115, 120]) - items = (unkeyed_items - | 'key' >> beam.Map( - lambda x: beam.window.TimestampedValue(('k', x), x))) - # [START setting_fixed_windows] - from apache_beam import window - fixed_windowed_items = ( - items | 'window' >> beam.WindowInto(window.FixedWindows(60))) - # [END setting_fixed_windows] - summed = (fixed_windowed_items - | 'group' >> beam.GroupByKey() - | 'combine' >> beam.CombineValues(sum)) - unkeyed = summed | 'unkey' >> beam.Map(lambda x: x[1]) - assert_that(unkeyed, equal_to([110, 215, 120])) - p.run() + with TestPipeline() as p: + unkeyed_items = p | beam.Create([22, 33, 55, 100, 115, 120]) + items = (unkeyed_items + | 'key' >> beam.Map( + lambda x: beam.window.TimestampedValue(('k', x), x))) + # [START setting_fixed_windows] + from apache_beam import window + fixed_windowed_items = ( + items | 'window' >> beam.WindowInto(window.FixedWindows(60))) + # [END setting_fixed_windows] + summed = (fixed_windowed_items + | 'group' >> beam.GroupByKey() + | 'combine' >> beam.CombineValues(sum)) + unkeyed = summed | 'unkey' >> beam.Map(lambda x: x[1]) + assert_that(unkeyed, equal_to([110, 215, 120])) def test_setting_sliding_windows(self): - p = TestPipeline() - unkeyed_items = p | beam.Create([2, 16, 23]) - items = (unkeyed_items - | 'key' >> beam.Map( - lambda x: beam.window.TimestampedValue(('k', x), x))) - # [START setting_sliding_windows] - from apache_beam import window - sliding_windowed_items = ( - items | 'window' >> beam.WindowInto(window.SlidingWindows(30, 5))) - # [END setting_sliding_windows] - summed = (sliding_windowed_items - | 'group' >> beam.GroupByKey() - | 'combine' >> beam.CombineValues(sum)) - unkeyed = summed | 'unkey' >> beam.Map(lambda x: x[1]) - assert_that(unkeyed, - equal_to([2, 2, 2, 18, 23, 39, 39, 39, 41, 41])) - p.run() + with TestPipeline() as p: + unkeyed_items = p | beam.Create([2, 16, 23]) + items = (unkeyed_items + | 'key' >> beam.Map( + lambda x: beam.window.TimestampedValue(('k', x), x))) + # [START setting_sliding_windows] + from apache_beam import window + sliding_windowed_items = ( + items | 'window' >> beam.WindowInto(window.SlidingWindows(30, 5))) + # [END setting_sliding_windows] + summed = (sliding_windowed_items + | 'group' >> beam.GroupByKey() + | 'combine' >> beam.CombineValues(sum)) + unkeyed = summed | 'unkey' >> beam.Map(lambda x: x[1]) + assert_that(unkeyed, + equal_to([2, 2, 2, 18, 23, 39, 39, 39, 41, 41])) def test_setting_session_windows(self): - p = TestPipeline() - unkeyed_items = p | beam.Create([2, 11, 16, 27]) - items = (unkeyed_items - | 'key' >> beam.Map( - lambda x: beam.window.TimestampedValue(('k', x), x))) - # [START setting_session_windows] - from apache_beam import window - session_windowed_items = ( - items | 'window' >> beam.WindowInto(window.Sessions(10))) - # [END setting_session_windows] - summed = (session_windowed_items - | 'group' >> beam.GroupByKey() - | 'combine' >> beam.CombineValues(sum)) - unkeyed = summed | 'unkey' >> beam.Map(lambda x: x[1]) - assert_that(unkeyed, - equal_to([29, 27])) - p.run() + with TestPipeline() as p: + unkeyed_items = p | beam.Create([2, 11, 16, 27]) + items = (unkeyed_items + | 'key' >> beam.Map( + lambda x: beam.window.TimestampedValue(('k', x), x))) + # [START setting_session_windows] + from apache_beam import window + session_windowed_items = ( + items | 'window' >> beam.WindowInto(window.Sessions(10))) + # [END setting_session_windows] + summed = (session_windowed_items + | 'group' >> beam.GroupByKey() + | 'combine' >> beam.CombineValues(sum)) + unkeyed = summed | 'unkey' >> beam.Map(lambda x: x[1]) + assert_that(unkeyed, + equal_to([29, 27])) def test_setting_global_window(self): - p = TestPipeline() - unkeyed_items = p | beam.Create([2, 11, 16, 27]) - items = (unkeyed_items - | 'key' >> beam.Map( - lambda x: beam.window.TimestampedValue(('k', x), x))) - # [START setting_global_window] - from apache_beam import window - session_windowed_items = ( - items | 'window' >> beam.WindowInto(window.GlobalWindows())) - # [END setting_global_window] - summed = (session_windowed_items - | 'group' >> beam.GroupByKey() - | 'combine' >> beam.CombineValues(sum)) - unkeyed = summed | 'unkey' >> beam.Map(lambda x: x[1]) - assert_that(unkeyed, equal_to([56])) - p.run() + with TestPipeline() as p: + unkeyed_items = p | beam.Create([2, 11, 16, 27]) + items = (unkeyed_items + | 'key' >> beam.Map( + lambda x: beam.window.TimestampedValue(('k', x), x))) + # [START setting_global_window] + from apache_beam import window + session_windowed_items = ( + items | 'window' >> beam.WindowInto(window.GlobalWindows())) + # [END setting_global_window] + summed = (session_windowed_items + | 'group' >> beam.GroupByKey() + | 'combine' >> beam.CombineValues(sum)) + unkeyed = summed | 'unkey' >> beam.Map(lambda x: x[1]) + assert_that(unkeyed, equal_to([56])) def test_setting_timestamp(self): - p = TestPipeline() - unkeyed_items = p | beam.Create([12, 30, 60, 61, 66]) - items = (unkeyed_items | 'key' >> beam.Map(lambda x: ('k', x))) + with TestPipeline() as p: + unkeyed_items = p | beam.Create([12, 30, 60, 61, 66]) + items = (unkeyed_items | 'key' >> beam.Map(lambda x: ('k', x))) - def extract_timestamp_from_log_entry(entry): - return entry[1] + def extract_timestamp_from_log_entry(entry): + return entry[1] - # [START setting_timestamp] - class AddTimestampDoFn(beam.DoFn): + # [START setting_timestamp] + class AddTimestampDoFn(beam.DoFn): - def process(self, element): - # Extract the numeric Unix seconds-since-epoch timestamp to be - # associated with the current log entry. - unix_timestamp = extract_timestamp_from_log_entry(element) - # Wrap and emit the current entry and new timestamp in a - # TimestampedValue. - yield beam.window.TimestampedValue(element, unix_timestamp) - - timestamped_items = items | 'timestamp' >> beam.ParDo(AddTimestampDoFn()) - # [END setting_timestamp] - fixed_windowed_items = ( - timestamped_items | 'window' >> beam.WindowInto( - beam.window.FixedWindows(60))) - summed = (fixed_windowed_items - | 'group' >> beam.GroupByKey() - | 'combine' >> beam.CombineValues(sum)) - unkeyed = summed | 'unkey' >> beam.Map(lambda x: x[1]) - assert_that(unkeyed, equal_to([42, 187])) - p.run() + def process(self, element): + # Extract the numeric Unix seconds-since-epoch timestamp to be + # associated with the current log entry. + unix_timestamp = extract_timestamp_from_log_entry(element) + # Wrap and emit the current entry and new timestamp in a + # TimestampedValue. + yield beam.window.TimestampedValue(element, unix_timestamp) + + timestamped_items = items | 'timestamp' >> beam.ParDo(AddTimestampDoFn()) + # [END setting_timestamp] + fixed_windowed_items = ( + timestamped_items | 'window' >> beam.WindowInto( + beam.window.FixedWindows(60))) + summed = (fixed_windowed_items + | 'group' >> beam.GroupByKey() + | 'combine' >> beam.CombineValues(sum)) + unkeyed = summed | 'unkey' >> beam.Map(lambda x: x[1]) + assert_that(unkeyed, equal_to([42, 187])) class PTransformTest(unittest.TestCase): @@ -919,10 +910,9 @@ def expand(self, pcoll): return pcoll | beam.Map(lambda x: len(x)) # [END model_composite_transform] - p = TestPipeline() - lengths = p | beam.Create(["a", "ab", "abc"]) | ComputeWordLengths() - assert_that(lengths, equal_to([1, 2, 3])) - p.run() + with TestPipeline() as p: + lengths = p | beam.Create(["a", "ab", "abc"]) | ComputeWordLengths() + assert_that(lengths, equal_to([1, 2, 3])) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/streaming_wordcap.py b/sdks/python/apache_beam/examples/streaming_wordcap.py index d0cc8a201b41..ce43e1f09849 100644 --- a/sdks/python/apache_beam/examples/streaming_wordcap.py +++ b/sdks/python/apache_beam/examples/streaming_wordcap.py @@ -41,22 +41,20 @@ def run(argv=None): help='Output PubSub topic of the form "/topics//".') known_args, pipeline_args = parser.parse_known_args(argv) - p = beam.Pipeline(argv=pipeline_args) + with beam.Pipeline(argv=pipeline_args) as p: - # Read the text file[pattern] into a PCollection. - lines = p | beam.io.Read( - beam.io.PubSubSource(known_args.input_topic)) + # Read the text file[pattern] into a PCollection. + lines = p | beam.io.Read( + beam.io.PubSubSource(known_args.input_topic)) - # Capitalize the characters in each line. - transformed = (lines - | 'capitalize' >> (beam.Map(lambda x: x.upper()))) + # Capitalize the characters in each line. + transformed = (lines + | 'capitalize' >> (beam.Map(lambda x: x.upper()))) - # Write to PubSub. - # pylint: disable=expression-not-assigned - transformed | beam.io.Write( - beam.io.PubSubSink(known_args.output_topic)) - - p.run().wait_until_finish() + # Write to PubSub. + # pylint: disable=expression-not-assigned + transformed | beam.io.Write( + beam.io.PubSubSink(known_args.output_topic)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/streaming_wordcount.py b/sdks/python/apache_beam/examples/streaming_wordcount.py index 4b6aecc98c4a..e9d5dbefa835 100644 --- a/sdks/python/apache_beam/examples/streaming_wordcount.py +++ b/sdks/python/apache_beam/examples/streaming_wordcount.py @@ -44,29 +44,27 @@ def run(argv=None): help='Output PubSub topic of the form "/topics//".') known_args, pipeline_args = parser.parse_known_args(argv) - p = beam.Pipeline(argv=pipeline_args) - - # Read the text file[pattern] into a PCollection. - lines = p | 'read' >> beam.io.Read( - beam.io.PubSubSource(known_args.input_topic)) - - # Capitalize the characters in each line. - transformed = (lines - | 'Split' >> ( - beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) - .with_output_types(unicode)) - | 'PairWithOne' >> beam.Map(lambda x: (x, 1)) - | beam.WindowInto(window.FixedWindows(15, 0)) - | 'Group' >> beam.GroupByKey() - | 'Count' >> beam.Map(lambda (word, ones): (word, sum(ones))) - | 'Format' >> beam.Map(lambda tup: '%s: %d' % tup)) - - # Write to PubSub. - # pylint: disable=expression-not-assigned - transformed | 'pubsub_write' >> beam.io.Write( - beam.io.PubSubSink(known_args.output_topic)) - - p.run().wait_until_finish() + with beam.Pipeline(argv=pipeline_args) as p: + + # Read the text file[pattern] into a PCollection. + lines = p | 'read' >> beam.io.Read( + beam.io.PubSubSource(known_args.input_topic)) + + # Capitalize the characters in each line. + transformed = (lines + | 'Split' >> ( + beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) + .with_output_types(unicode)) + | 'PairWithOne' >> beam.Map(lambda x: (x, 1)) + | beam.WindowInto(window.FixedWindows(15, 0)) + | 'Group' >> beam.GroupByKey() + | 'Count' >> beam.Map(lambda (word, ones): (word, sum(ones))) + | 'Format' >> beam.Map(lambda tup: '%s: %d' % tup)) + + # Write to PubSub. + # pylint: disable=expression-not-assigned + transformed | 'pubsub_write' >> beam.io.Write( + beam.io.PubSubSink(known_args.output_topic)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/wordcount.py b/sdks/python/apache_beam/examples/wordcount.py index e7e542a74219..34dedb2b819a 100644 --- a/sdks/python/apache_beam/examples/wordcount.py +++ b/sdks/python/apache_beam/examples/wordcount.py @@ -102,7 +102,6 @@ def run(argv=None): # pylint: disable=expression-not-assigned output | 'write' >> WriteToText(known_args.output) - # Actually run the pipeline (all operations above are deferred). result = p.run() result.wait_until_finish() diff --git a/sdks/python/apache_beam/examples/wordcount_debugging.py b/sdks/python/apache_beam/examples/wordcount_debugging.py index ca9f7b6371de..c0ffd356364c 100644 --- a/sdks/python/apache_beam/examples/wordcount_debugging.py +++ b/sdks/python/apache_beam/examples/wordcount_debugging.py @@ -118,35 +118,32 @@ def run(argv=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) - - # Read the text file[pattern] into a PCollection, count the occurrences of - # each word and filter by a list of words. - filtered_words = ( - p | 'read' >> ReadFromText(known_args.input) - | CountWords() - | 'FilterText' >> beam.ParDo(FilterTextFn('Flourish|stomach'))) - - # assert_that is a convenient PTransform that checks a PCollection has an - # expected value. Asserts are best used in unit tests with small data sets but - # is demonstrated here as a teaching tool. - # - # Note assert_that does not provide any output and that successful completion - # of the Pipeline implies that the expectations were met. Learn more at - # https://cloud.google.com/dataflow/pipelines/testing-your-pipeline on how to - # test your pipeline. - assert_that( - filtered_words, equal_to([('Flourish', 3), ('stomach', 1)])) - - # Format the counts into a PCollection of strings and write the output using a - # "Write" transform that has side effects. - # pylint: disable=unused-variable - output = (filtered_words - | 'format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) - | 'write' >> WriteToText(known_args.output)) - - # Actually run the pipeline (all operations above are deferred). - p.run().wait_until_finish() + with beam.Pipeline(options=pipeline_options) as p: + + # Read the text file[pattern] into a PCollection, count the occurrences of + # each word and filter by a list of words. + filtered_words = ( + p | 'read' >> ReadFromText(known_args.input) + | CountWords() + | 'FilterText' >> beam.ParDo(FilterTextFn('Flourish|stomach'))) + + # assert_that is a convenient PTransform that checks a PCollection has an + # expected value. Asserts are best used in unit tests with small data sets + # but is demonstrated here as a teaching tool. + # + # Note assert_that does not provide any output and that successful + # completion of the Pipeline implies that the expectations were met. Learn + # more at https://cloud.google.com/dataflow/pipelines/testing-your-pipeline + # on how to best test your pipeline. + assert_that( + filtered_words, equal_to([('Flourish', 3), ('stomach', 1)])) + + # Format the counts into a PCollection of strings and write the output using + # a "Write" transform that has side effects. + # pylint: disable=unused-variable + output = (filtered_words + | 'format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) + | 'write' >> WriteToText(known_args.output)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/wordcount_minimal.py b/sdks/python/apache_beam/examples/wordcount_minimal.py index 5109c089f993..76b0a221df7a 100644 --- a/sdks/python/apache_beam/examples/wordcount_minimal.py +++ b/sdks/python/apache_beam/examples/wordcount_minimal.py @@ -92,28 +92,25 @@ def run(argv=None): # workflow rely on global context (e.g., a module imported at module level). pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True - p = beam.Pipeline(options=pipeline_options) + with beam.Pipeline(options=pipeline_options) as p: - # Read the text file[pattern] into a PCollection. - lines = p | 'read' >> ReadFromText(known_args.input) + # Read the text file[pattern] into a PCollection. + lines = p | ReadFromText(known_args.input) - # Count the occurrences of each word. - counts = (lines - | 'split' >> (beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) - .with_output_types(unicode)) - | 'pair_with_one' >> beam.Map(lambda x: (x, 1)) - | 'group' >> beam.GroupByKey() - | 'count' >> beam.Map(lambda (word, ones): (word, sum(ones)))) + # Count the occurrences of each word. + counts = ( + lines + | 'Split' >> (beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x)) + .with_output_types(unicode)) + | 'PairWithOne' >> beam.Map(lambda x: (x, 1)) + | 'GroupAndSum' >> beam.CombinePerKey(sum)) - # Format the counts into a PCollection of strings. - output = counts | 'format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) + # Format the counts into a PCollection of strings. + output = counts | 'Format' >> beam.Map(lambda (w, c): '%s: %s' % (w, c)) - # Write the output using a "Write" transform that has side effects. - # pylint: disable=expression-not-assigned - output | 'write' >> WriteToText(known_args.output) - - # Actually run the pipeline (all operations above are deferred). - p.run().wait_until_finish() + # Write the output using a "Write" transform that has side effects. + # pylint: disable=expression-not-assigned + output | WriteToText(known_args.output) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/io/filebasedsink_test.py b/sdks/python/apache_beam/io/filebasedsink_test.py index 1f6aeee56677..7c8ddb4072ef 100644 --- a/sdks/python/apache_beam/io/filebasedsink_test.py +++ b/sdks/python/apache_beam/io/filebasedsink_test.py @@ -146,9 +146,8 @@ def test_empty_write(self): sink = MyFileBasedSink( temp_path, file_name_suffix='.output', coder=coders.ToStringCoder() ) - p = TestPipeline() - p | beam.Create([]) | beam.io.Write(sink) # pylint: disable=expression-not-assigned - p.run() + with TestPipeline() as p: + p | beam.Create([]) | beam.io.Write(sink) # pylint: disable=expression-not-assigned self.assertEqual( open(temp_path + '-00000-of-00001.output').read(), '[start][end]') @@ -160,9 +159,8 @@ def test_static_value_provider_empty_write(self): file_name_suffix=StaticValueProvider(value_type=str, value='.output'), coder=coders.ToStringCoder() ) - p = TestPipeline() - p | beam.Create([]) | beam.io.Write(sink) # pylint: disable=expression-not-assigned - p.run() + with TestPipeline() as p: + p | beam.Create([]) | beam.io.Write(sink) # pylint: disable=expression-not-assigned self.assertEqual( open(temp_path.get() + '-00000-of-00001.output').read(), '[start][end]') @@ -174,10 +172,8 @@ def test_fixed_shard_write(self): num_shards=3, shard_name_template='_NN_SSS_', coder=coders.ToStringCoder()) - p = TestPipeline() - p | beam.Create(['a', 'b']) | beam.io.Write(sink) # pylint: disable=expression-not-assigned - - p.run() + with TestPipeline() as p: + p | beam.Create(['a', 'b']) | beam.io.Write(sink) # pylint: disable=expression-not-assigned concat = ''.join( open(temp_path + '_03_%03d_.output' % shard_num).read() diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 5048534b3852..9093abfccfc3 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -28,19 +28,18 @@ Typical usage: # Create a pipeline object using a local runner for execution. - p = beam.Pipeline('DirectRunner') + with beam.Pipeline('DirectRunner') as p: - # Add to the pipeline a "Create" transform. When executed this - # transform will produce a PCollection object with the specified values. - pcoll = p | 'Create' >> beam.Create([1, 2, 3]) + # Add to the pipeline a "Create" transform. When executed this + # transform will produce a PCollection object with the specified values. + pcoll = p | 'Create' >> beam.Create([1, 2, 3]) - # Another transform could be applied to pcoll, e.g., writing to a text file. - # For other transforms, refer to transforms/ directory. - pcoll | 'Write' >> beam.io.WriteToText('./output') + # Another transform could be applied to pcoll, e.g., writing to a text file. + # For other transforms, refer to transforms/ directory. + pcoll | 'Write' >> beam.io.WriteToText('./output') - # run() will execute the DAG stored in the pipeline. The execution of the - # nodes visited is done using the specified local runner. - p.run() + # run() will execute the DAG stored in the pipeline. The execution of the + # nodes visited is done using the specified local runner. """ diff --git a/sdks/python/apache_beam/transforms/combiners_test.py b/sdks/python/apache_beam/transforms/combiners_test.py index 946a60a3675c..c79fec864acb 100644 --- a/sdks/python/apache_beam/transforms/combiners_test.py +++ b/sdks/python/apache_beam/transforms/combiners_test.py @@ -247,26 +247,23 @@ def match(actual): pipeline.run() def test_tuple_combine_fn(self): - p = TestPipeline() - result = ( - p - | Create([('a', 100, 0.0), ('b', 10, -1), ('c', 1, 100)]) - | beam.CombineGlobally(combine.TupleCombineFn(max, - combine.MeanCombineFn(), - sum)).without_defaults()) - assert_that(result, equal_to([('c', 111.0 / 3, 99.0)])) - p.run() + with TestPipeline() as p: + result = ( + p + | Create([('a', 100, 0.0), ('b', 10, -1), ('c', 1, 100)]) + | beam.CombineGlobally(combine.TupleCombineFn( + max, combine.MeanCombineFn(), sum)).without_defaults()) + assert_that(result, equal_to([('c', 111.0 / 3, 99.0)])) def test_tuple_combine_fn_without_defaults(self): - p = TestPipeline() - result = ( - p - | Create([1, 1, 2, 3]) - | beam.CombineGlobally( - combine.TupleCombineFn(min, combine.MeanCombineFn(), max) - .with_common_input()).without_defaults()) - assert_that(result, equal_to([(1, 7.0 / 4, 3)])) - p.run() + with TestPipeline() as p: + result = ( + p + | Create([1, 1, 2, 3]) + | beam.CombineGlobally( + combine.TupleCombineFn(min, combine.MeanCombineFn(), max) + .with_common_input()).without_defaults()) + assert_that(result, equal_to([(1, 7.0 / 4, 3)])) def test_to_list_and_to_dict(self): pipeline = TestPipeline() @@ -295,29 +292,26 @@ def match(actual): pipeline.run() def test_combine_globally_with_default(self): - p = TestPipeline() - assert_that(p | Create([]) | CombineGlobally(sum), equal_to([0])) - p.run() + with TestPipeline() as p: + assert_that(p | Create([]) | CombineGlobally(sum), equal_to([0])) def test_combine_globally_without_default(self): - p = TestPipeline() - result = p | Create([]) | CombineGlobally(sum).without_defaults() - assert_that(result, equal_to([])) - p.run() + with TestPipeline() as p: + result = p | Create([]) | CombineGlobally(sum).without_defaults() + assert_that(result, equal_to([])) def test_combine_globally_with_default_side_input(self): - class CombineWithSideInput(PTransform): + class SideInputCombine(PTransform): def expand(self, pcoll): side = pcoll | CombineGlobally(sum).as_singleton_view() main = pcoll.pipeline | Create([None]) return main | Map(lambda _, s: s, side) - p = TestPipeline() - result1 = p | 'i1' >> Create([]) | 'c1' >> CombineWithSideInput() - result2 = p | 'i2' >> Create([1, 2, 3, 4]) | 'c2' >> CombineWithSideInput() - assert_that(result1, equal_to([0]), label='r1') - assert_that(result2, equal_to([10]), label='r2') - p.run() + with TestPipeline() as p: + result1 = p | 'i1' >> Create([]) | 'c1' >> SideInputCombine() + result2 = p | 'i2' >> Create([1, 2, 3, 4]) | 'c2' >> SideInputCombine() + assert_that(result1, equal_to([0]), label='r1') + assert_that(result2, equal_to([10]), label='r2') if __name__ == '__main__': diff --git a/sdks/python/apache_beam/transforms/window_test.py b/sdks/python/apache_beam/transforms/window_test.py index fd1bb9d5250b..977a364ad883 100644 --- a/sdks/python/apache_beam/transforms/window_test.py +++ b/sdks/python/apache_beam/transforms/window_test.py @@ -167,90 +167,85 @@ def timestamped_key_values(self, pipeline, key, *timestamps): | Map(lambda x: WindowedValue((key, x), x, [GlobalWindow()]))) def test_sliding_windows(self): - p = TestPipeline() - pcoll = self.timestamped_key_values(p, 'key', 1, 2, 3) - result = (pcoll - | 'w' >> WindowInto(SlidingWindows(period=2, size=4)) - | GroupByKey() - | reify_windows) - expected = [('key @ [-2.0, 2.0)', [1]), - ('key @ [0.0, 4.0)', [1, 2, 3]), - ('key @ [2.0, 6.0)', [2, 3])] - assert_that(result, equal_to(expected)) - p.run() + with TestPipeline() as p: + pcoll = self.timestamped_key_values(p, 'key', 1, 2, 3) + result = (pcoll + | 'w' >> WindowInto(SlidingWindows(period=2, size=4)) + | GroupByKey() + | reify_windows) + expected = [('key @ [-2.0, 2.0)', [1]), + ('key @ [0.0, 4.0)', [1, 2, 3]), + ('key @ [2.0, 6.0)', [2, 3])] + assert_that(result, equal_to(expected)) def test_sessions(self): - p = TestPipeline() - pcoll = self.timestamped_key_values(p, 'key', 1, 2, 3, 20, 35, 27) - result = (pcoll - | 'w' >> WindowInto(Sessions(10)) - | GroupByKey() - | sort_values - | reify_windows) - expected = [('key @ [1.0, 13.0)', [1, 2, 3]), - ('key @ [20.0, 45.0)', [20, 27, 35])] - assert_that(result, equal_to(expected)) - p.run() + with TestPipeline() as p: + pcoll = self.timestamped_key_values(p, 'key', 1, 2, 3, 20, 35, 27) + result = (pcoll + | 'w' >> WindowInto(Sessions(10)) + | GroupByKey() + | sort_values + | reify_windows) + expected = [('key @ [1.0, 13.0)', [1, 2, 3]), + ('key @ [20.0, 45.0)', [20, 27, 35])] + assert_that(result, equal_to(expected)) def test_timestamped_value(self): - p = TestPipeline() - result = (p - | 'start' >> Create([(k, k) for k in range(10)]) - | Map(lambda (x, t): TimestampedValue(x, t)) - | 'w' >> WindowInto(FixedWindows(5)) - | Map(lambda v: ('key', v)) - | GroupByKey()) - assert_that(result, equal_to([('key', [0, 1, 2, 3, 4]), - ('key', [5, 6, 7, 8, 9])])) - p.run() + with TestPipeline() as p: + result = (p + | 'start' >> Create([(k, k) for k in range(10)]) + | Map(lambda (x, t): TimestampedValue(x, t)) + | 'w' >> WindowInto(FixedWindows(5)) + | Map(lambda v: ('key', v)) + | GroupByKey()) + assert_that(result, equal_to([('key', [0, 1, 2, 3, 4]), + ('key', [5, 6, 7, 8, 9])])) def test_rewindow(self): - p = TestPipeline() - result = (p - | Create([(k, k) for k in range(10)]) - | Map(lambda (x, t): TimestampedValue(x, t)) - | 'window' >> WindowInto(SlidingWindows(period=2, size=6)) - # Per the model, each element is now duplicated across - # three windows. Rewindowing must preserve this duplication. - | 'rewindow' >> WindowInto(FixedWindows(5)) - | 'rewindow2' >> WindowInto(FixedWindows(5)) - | Map(lambda v: ('key', v)) - | GroupByKey()) - assert_that(result, equal_to([('key', sorted([0, 1, 2, 3, 4] * 3)), - ('key', sorted([5, 6, 7, 8, 9] * 3))])) - p.run() + with TestPipeline() as p: + result = (p + | Create([(k, k) for k in range(10)]) + | Map(lambda (x, t): TimestampedValue(x, t)) + | 'window' >> WindowInto(SlidingWindows(period=2, size=6)) + # Per the model, each element is now duplicated across + # three windows. Rewindowing must preserve this duplication. + | 'rewindow' >> WindowInto(FixedWindows(5)) + | 'rewindow2' >> WindowInto(FixedWindows(5)) + | Map(lambda v: ('key', v)) + | GroupByKey()) + assert_that(result, equal_to([('key', sorted([0, 1, 2, 3, 4] * 3)), + ('key', sorted([5, 6, 7, 8, 9] * 3))])) def test_timestamped_with_combiners(self): - p = TestPipeline() - result = (p - # Create some initial test values. - | 'start' >> Create([(k, k) for k in range(10)]) - # The purpose of the WindowInto transform is to establish a - # FixedWindows windowing function for the PCollection. - # It does not bucket elements into windows since the timestamps - # from Create are not spaced 5 ms apart and very likely they all - # fall into the same window. - | 'w' >> WindowInto(FixedWindows(5)) - # Generate timestamped values using the values as timestamps. - # Now there are values 5 ms apart and since Map propagates the - # windowing function from input to output the output PCollection - # will have elements falling into different 5ms windows. - | Map(lambda (x, t): TimestampedValue(x, t)) - # We add a 'key' to each value representing the index of the - # window. This is important since there is no guarantee of - # order for the elements of a PCollection. - | Map(lambda v: (v / 5, v))) - # Sum all elements associated with a key and window. Although it - # is called CombinePerKey it is really CombinePerKeyAndWindow the - # same way GroupByKey is really GroupByKeyAndWindow. - sum_per_window = result | CombinePerKey(sum) - # Compute mean per key and window. - mean_per_window = result | combiners.Mean.PerKey() - assert_that(sum_per_window, equal_to([(0, 10), (1, 35)]), - label='assert:sum') - assert_that(mean_per_window, equal_to([(0, 2.0), (1, 7.0)]), - label='assert:mean') - p.run() + with TestPipeline() as p: + result = (p + # Create some initial test values. + | 'start' >> Create([(k, k) for k in range(10)]) + # The purpose of the WindowInto transform is to establish a + # FixedWindows windowing function for the PCollection. + # It does not bucket elements into windows since the timestamps + # from Create are not spaced 5 ms apart and very likely they all + # fall into the same window. + | 'w' >> WindowInto(FixedWindows(5)) + # Generate timestamped values using the values as timestamps. + # Now there are values 5 ms apart and since Map propagates the + # windowing function from input to output the output PCollection + # will have elements falling into different 5ms windows. + | Map(lambda (x, t): TimestampedValue(x, t)) + # We add a 'key' to each value representing the index of the + # window. This is important since there is no guarantee of + # order for the elements of a PCollection. + | Map(lambda v: (v / 5, v))) + # Sum all elements associated with a key and window. Although it + # is called CombinePerKey it is really CombinePerKeyAndWindow the + # same way GroupByKey is really GroupByKeyAndWindow. + sum_per_window = result | CombinePerKey(sum) + # Compute mean per key and window. + mean_per_window = result | combiners.Mean.PerKey() + assert_that(sum_per_window, equal_to([(0, 10), (1, 35)]), + label='assert:sum') + assert_that(mean_per_window, equal_to([(0, 2.0), (1, 7.0)]), + label='assert:mean') class RunnerApiTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/transforms/write_ptransform_test.py b/sdks/python/apache_beam/transforms/write_ptransform_test.py index e31b9cc9830d..50f0debb0a70 100644 --- a/sdks/python/apache_beam/transforms/write_ptransform_test.py +++ b/sdks/python/apache_beam/transforms/write_ptransform_test.py @@ -98,11 +98,10 @@ def _run_write_test(self, return_write_results=True): write_to_test_sink = WriteToTestSink(return_init_result, return_write_results) - p = TestPipeline() - result = p | beam.Create(data) | write_to_test_sink | beam.Map(list) + with TestPipeline() as p: + result = p | beam.Create(data) | write_to_test_sink | beam.Map(list) - assert_that(result, is_empty()) - p.run() + assert_that(result, is_empty()) sink = write_to_test_sink.last_sink self.assertIsNotNone(sink) diff --git a/sdks/python/apache_beam/typehints/typed_pipeline_test.py b/sdks/python/apache_beam/typehints/typed_pipeline_test.py index 589dc0e5ac82..c81ef320e027 100644 --- a/sdks/python/apache_beam/typehints/typed_pipeline_test.py +++ b/sdks/python/apache_beam/typehints/typed_pipeline_test.py @@ -168,12 +168,11 @@ def test_deferred_side_inputs(self): @typehints.with_input_types(str, int) def repeat(s, times): return s * times - p = TestPipeline() - main_input = p | beam.Create(['a', 'bb', 'c']) - side_input = p | 'side' >> beam.Create([3]) - result = main_input | beam.Map(repeat, pvalue.AsSingleton(side_input)) - assert_that(result, equal_to(['aaa', 'bbbbbb', 'ccc'])) - p.run() + with TestPipeline() as p: + main_input = p | beam.Create(['a', 'bb', 'c']) + side_input = p | 'side' >> beam.Create([3]) + result = main_input | beam.Map(repeat, pvalue.AsSingleton(side_input)) + assert_that(result, equal_to(['aaa', 'bbbbbb', 'ccc'])) bad_side_input = p | 'bad_side' >> beam.Create(['z']) with self.assertRaises(typehints.TypeCheckError): @@ -183,12 +182,11 @@ def test_deferred_side_input_iterable(self): @typehints.with_input_types(str, typehints.Iterable[str]) def concat(glue, items): return glue.join(sorted(items)) - p = TestPipeline() - main_input = p | beam.Create(['a', 'bb', 'c']) - side_input = p | 'side' >> beam.Create(['x', 'y', 'z']) - result = main_input | beam.Map(concat, pvalue.AsIter(side_input)) - assert_that(result, equal_to(['xayaz', 'xbbybbz', 'xcycz'])) - p.run() + with TestPipeline() as p: + main_input = p | beam.Create(['a', 'bb', 'c']) + side_input = p | 'side' >> beam.Create(['x', 'y', 'z']) + result = main_input | beam.Map(concat, pvalue.AsIter(side_input)) + assert_that(result, equal_to(['xayaz', 'xbbybbz', 'xcycz'])) bad_side_input = p | 'bad_side' >> beam.Create([1, 2, 3]) with self.assertRaises(typehints.TypeCheckError): From 98a75551064c742d108d8c5ec8fc0783db7761d2 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 15:28:44 -0700 Subject: [PATCH 038/159] Move StepContext to top level --- .../translation/utils/NoOpStepContext.java | 6 +- .../runners/core/BaseExecutionContext.java | 8 +-- .../apache/beam/runners/core/DoFnRunners.java | 1 - .../beam/runners/core/ExecutionContext.java | 47 ------------- .../beam/runners/core/SimpleDoFnRunner.java | 1 - .../apache/beam/runners/core/StepContext.java | 70 +++++++++++++++++++ .../functions/FlinkNoOpStepContext.java | 2 +- .../wrappers/streaming/DoFnOperator.java | 7 +- .../translation/SparkProcessContext.java | 2 +- .../beam/fn/harness/fake/FakeStepContext.java | 2 +- 10 files changed, 83 insertions(+), 63 deletions(-) create mode 100644 runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java index 721eecd3d626..241a9856736d 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java @@ -19,8 +19,8 @@ import java.io.IOException; import java.io.Serializable; -import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -28,9 +28,9 @@ import org.apache.beam.sdk.values.TupleTag; /** - * Serializable {@link ExecutionContext.StepContext} that does nothing. + * Serializable {@link StepContext} that does nothing. */ -public class NoOpStepContext implements ExecutionContext.StepContext, Serializable { +public class NoOpStepContext implements StepContext, Serializable { private static final long serialVersionUID = 1L; @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java index 23d61f89b1db..ed3714372b47 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java @@ -49,7 +49,7 @@ * {@link #getOrCreateStepContext(String, String)}, and {@link #getAllStepContexts()} * will be appropriately specialized. */ -public abstract class BaseExecutionContext +public abstract class BaseExecutionContext implements ExecutionContext { private Map cachedStepContexts = new LinkedHashMap<>(); @@ -81,7 +81,7 @@ public T create() { * Factory method interface to create an execution context if none exists during * {@link #getOrCreateStepContext(String, CreateStepContextFunction)}. */ - protected interface CreateStepContextFunction { + protected interface CreateStepContextFunction { T create(); } @@ -111,12 +111,12 @@ public void noteOutput(WindowedValue output) {} public void noteOutput(TupleTag tag, WindowedValue output) {} /** - * Base class for implementations of {@link ExecutionContext.StepContext}. + * Base class for implementations of {@link org.apache.beam.runners.core.StepContext}. * *

To complete a concrete subclass, implement {@link #timerInternals} and * {@link #stateInternals}. */ - public abstract static class StepContext implements ExecutionContext.StepContext { + public abstract static class StepContext implements org.apache.beam.runners.core.StepContext { private final ExecutionContext executionContext; private final String stepName; private final String transformName; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java index 71dfd11a496d..9d3e25dbb310 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java @@ -19,7 +19,6 @@ import java.util.Collection; import java.util.List; -import org.apache.beam.runners.core.ExecutionContext.StepContext; import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.ProcessFn; import org.apache.beam.runners.core.StatefulDoFnRunner.CleanupTimer; import org.apache.beam.runners.core.StatefulDoFnRunner.StateCleaner; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java index d2fdaac50982..f431c92859ec 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java @@ -17,11 +17,8 @@ */ package org.apache.beam.runners.core; -import java.io.IOException; import java.util.Collection; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn.WindowedContext; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; @@ -52,48 +49,4 @@ public interface ExecutionContext { */ void noteOutput(TupleTag tag, WindowedValue output); - /** - * Per-step, per-key context used for retrieving state. - */ - public interface StepContext { - - /** - * The name of the step. - */ - String getStepName(); - - /** - * The name of the transform for the step. - */ - String getTransformName(); - - /** - * Hook for subclasses to implement that will be called whenever - * {@link WindowedContext#output} - * is called. - */ - void noteOutput(WindowedValue output); - - /** - * Hook for subclasses to implement that will be called whenever - * {@link WindowedContext#output} - * is called. - */ - void noteOutput(TupleTag tag, WindowedValue output); - - /** - * Writes the given {@code PCollectionView} data to a globally accessible location. - */ - void writePCollectionViewData( - TupleTag tag, - Iterable> data, - Coder>> dataCoder, - W window, - Coder windowCoder) - throws IOException; - - StateInternals stateInternals(); - - TimerInternals timerInternals(); - } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 65384da9f8f7..adbe62e4cc43 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -29,7 +29,6 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.runners.core.DoFnRunners.OutputManager; -import org.apache.beam.runners.core.ExecutionContext.StepContext; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.state.State; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java new file mode 100644 index 000000000000..a414830bae6d --- /dev/null +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core; + +import java.io.IOException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn.WindowedContext; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TupleTag; + +/** + * Per-step, per-key context used for retrieving state. + */ +public interface StepContext { + + /** + * The name of the step. + */ + String getStepName(); + + /** + * The name of the transform for the step. + */ + String getTransformName(); + + /** + * Hook for subclasses to implement that will be called whenever + * {@link WindowedContext#output} + * is called. + */ + void noteOutput(WindowedValue output); + + /** + * Hook for subclasses to implement that will be called whenever + * {@link WindowedContext#output} + * is called. + */ + void noteOutput(TupleTag tag, WindowedValue output); + + /** + * Writes the given {@code PCollectionView} data to a globally accessible location. + */ + void writePCollectionViewData( + TupleTag tag, + Iterable> data, + Coder>> dataCoder, + W window, + Coder windowCoder) + throws IOException; + + StateInternals stateInternals(); + + TimerInternals timerInternals(); +} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java index 86408016f19b..c394ebdf6f28 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.flink.translation.functions; import java.io.IOException; -import org.apache.beam.runners.core.ExecutionContext.StepContext; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.coders.Coder; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index f35ba7a0b086..c9f106a9079b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -32,7 +32,6 @@ import javax.annotation.Nullable; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; -import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetNewDoFn; import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; @@ -184,7 +183,7 @@ public DoFnOperator( TimerInternals.TimerDataCoder.of(windowingStrategy.getWindowFn().windowCoder()); } - private ExecutionContext.StepContext createStepContext() { + private org.apache.beam.runners.core.StepContext createStepContext() { return new StepContext(); } @@ -250,7 +249,7 @@ public void open() throws Exception { doFnInvoker.invokeSetup(); - ExecutionContext.StepContext stepContext = createStepContext(); + org.apache.beam.runners.core.StepContext stepContext = createStepContext(); doFnRunner = DoFnRunners.simpleRunner( serializedOptions.getPipelineOptions(), @@ -676,7 +675,7 @@ public void output(TupleTag tag, WindowedValue value) { * {@link StepContext} for running {@link DoFn DoFns} on Flink. This does not allow * accessing state or timer internals. */ - protected class StepContext implements ExecutionContext.StepContext { + protected class StepContext implements org.apache.beam.runners.core.StepContext { @Override public String getStepName() { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java index ffe343bc2d8b..91474223a7cb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java @@ -24,7 +24,7 @@ import java.util.Iterator; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners.OutputManager; -import org.apache.beam.runners.core.ExecutionContext.StepContext; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.coders.Coder; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java index 9b79d110bf37..b206bc7b8f85 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java @@ -19,7 +19,7 @@ package org.apache.beam.fn.harness.fake; import java.io.IOException; -import org.apache.beam.runners.core.ExecutionContext.StepContext; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.coders.Coder; From bed1c53fb47e2d623d6671ce69b82579992df642 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 15:30:33 -0700 Subject: [PATCH 039/159] Remove StepContext.noteOutput --- .../apex/translation/utils/NoOpStepContext.java | 8 -------- .../beam/runners/core/BaseExecutionContext.java | 16 ---------------- .../beam/runners/core/ExecutionContext.java | 16 ---------------- .../beam/runners/core/SimpleDoFnRunner.java | 6 ------ .../apache/beam/runners/core/StepContext.java | 15 --------------- .../functions/FlinkNoOpStepContext.java | 10 ---------- .../wrappers/streaming/DoFnOperator.java | 6 ------ .../spark/translation/SparkProcessContext.java | 6 ------ .../beam/fn/harness/fake/FakeStepContext.java | 8 -------- 9 files changed, 91 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java index 241a9856736d..51e843b56652 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java @@ -43,14 +43,6 @@ public String getTransformName() { return null; } - @Override - public void noteOutput(WindowedValue output) { - } - - @Override - public void noteOutput(TupleTag tag, WindowedValue output) { - } - @Override public void writePCollectionViewData(TupleTag tag, Iterable> data, diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java index ed3714372b47..a00699998bcd 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java @@ -104,12 +104,6 @@ public Collection getAllStepContexts() { return Collections.unmodifiableCollection(cachedStepContexts.values()); } - @Override - public void noteOutput(WindowedValue output) {} - - @Override - public void noteOutput(TupleTag tag, WindowedValue output) {} - /** * Base class for implementations of {@link org.apache.beam.runners.core.StepContext}. * @@ -137,16 +131,6 @@ public String getTransformName() { return transformName; } - @Override - public void noteOutput(WindowedValue output) { - executionContext.noteOutput(output); - } - - @Override - public void noteOutput(TupleTag tag, WindowedValue output) { - executionContext.noteOutput(tag, output); - } - @Override public void writePCollectionViewData( TupleTag tag, diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java index f431c92859ec..eac3599d9757 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java @@ -18,9 +18,6 @@ package org.apache.beam.runners.core; import java.util.Collection; -import org.apache.beam.sdk.transforms.DoFn.WindowedContext; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; /** * Context for the current execution. This is guaranteed to exist during processing, @@ -36,17 +33,4 @@ public interface ExecutionContext { * Returns a collection view of all of the {@link StepContext}s. */ Collection getAllStepContexts(); - - /** - * Hook for subclasses to implement that will be called whenever - * {@link WindowedContext#output(TupleTag, Object)} is called. - */ - void noteOutput(WindowedValue output); - - /** - * Hook for subclasses to implement that will be called whenever - * {@link WindowedContext#output(TupleTag, Object)} is called. - */ - void noteOutput(TupleTag tag, WindowedValue output); - } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index adbe62e4cc43..97b0b3307740 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -352,9 +352,6 @@ void outputWindowedValue( void outputWindowedValue(WindowedValue windowedElem) { outputManager.output(mainOutputTag, windowedElem); - if (stepContext != null) { - stepContext.noteOutput(windowedElem); - } } private void outputWindowedValue( @@ -380,9 +377,6 @@ private void outputWindowedValue(TupleTag tag, WindowedValue windowedE } outputManager.output(tag, windowedElem); - if (stepContext != null) { - stepContext.noteOutput(tag, windowedElem); - } } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java index a414830bae6d..fd2575d5f74b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java @@ -19,7 +19,6 @@ import java.io.IOException; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.DoFn.WindowedContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; @@ -39,20 +38,6 @@ public interface StepContext { */ String getTransformName(); - /** - * Hook for subclasses to implement that will be called whenever - * {@link WindowedContext#output} - * is called. - */ - void noteOutput(WindowedValue output); - - /** - * Hook for subclasses to implement that will be called whenever - * {@link WindowedContext#output} - * is called. - */ - void noteOutput(TupleTag tag, WindowedValue output); - /** * Writes the given {@code PCollectionView} data to a globally accessible location. */ diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java index c394ebdf6f28..d999494eee79 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java @@ -41,16 +41,6 @@ public String getTransformName() { return null; } - @Override - public void noteOutput(WindowedValue output) { - - } - - @Override - public void noteOutput(TupleTag tag, WindowedValue output) { - - } - @Override public void writePCollectionViewData( TupleTag tag, diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index c9f106a9079b..2bb9c2003898 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -687,12 +687,6 @@ public String getTransformName() { return null; } - @Override - public void noteOutput(WindowedValue output) {} - - @Override - public void noteOutput(TupleTag tag, WindowedValue output) {} - @Override public void writePCollectionViewData( TupleTag tag, diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java index 91474223a7cb..31e616cbf0f4 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java @@ -109,12 +109,6 @@ public String getTransformName() { return null; } - @Override - public void noteOutput(WindowedValue output) { } - - @Override - public void noteOutput(TupleTag tag, WindowedValue output) { } - @Override public void writePCollectionViewData( TupleTag tag, diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java index b206bc7b8f85..750c167d8993 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java @@ -41,14 +41,6 @@ public String getTransformName() { return "TODO"; } - @Override - public void noteOutput(WindowedValue output) { - } - - @Override - public void noteOutput(TupleTag tag, WindowedValue output) { - } - @Override public void writePCollectionViewData( TupleTag tag, From 8b7a1f6dfe0ac33814a0b0c67f37f47ab449ec4b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 15:34:37 -0700 Subject: [PATCH 040/159] Rename BaseExecutionContext.StepContext to BaseStepContext --- .../beam/runners/core/BaseExecutionContext.java | 14 +++++++------- .../beam/runners/core/SimpleDoFnRunnerTest.java | 5 +++-- .../beam/runners/core/StatefulDoFnRunnerTest.java | 5 +++-- .../runners/direct/DirectExecutionContext.java | 2 +- 4 files changed, 14 insertions(+), 12 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java index a00699998bcd..5667250b3288 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java @@ -31,11 +31,11 @@ * Base class for implementations of {@link ExecutionContext}. * *

A concrete subclass should implement {@link #createStepContext} to create the appropriate - * {@link StepContext} implementation. Any {@code StepContext} created will + * {@link BaseStepContext} implementation. Any {@code StepContext} created will * be cached for the lifetime of this {@link ExecutionContext}. * *

BaseExecutionContext is generic to allow implementing subclasses to return a concrete subclass - * of {@link StepContext} from {@link #getOrCreateStepContext(String, String)} and + * of {@link BaseStepContext} from {@link #getOrCreateStepContext(String, String)} and * {@link #getAllStepContexts()} without forcing each subclass to override the method, e.g. *

{@code
  * {@literal @}Override
@@ -56,12 +56,12 @@ public abstract class BaseExecutionContext
 
   /**
    * Implementations should override this to create the specific type
-   * of {@link StepContext} they need.
+   * of {@link BaseStepContext} they need.
    */
   protected abstract T createStepContext(String stepName, String transformName);
 
   /**
-   * Returns the {@link StepContext} associated with the given step.
+   * Returns the {@link BaseStepContext} associated with the given step.
    */
   @Override
   public T getOrCreateStepContext(String stepName, String transformName) {
@@ -97,7 +97,7 @@ protected final T getOrCreateStepContext(String stepName,
   }
 
   /**
-   * Returns a collection view of all of the {@link StepContext}s.
+   * Returns a collection view of all of the {@link BaseStepContext}s.
    */
   @Override
   public Collection getAllStepContexts() {
@@ -110,12 +110,12 @@ public Collection getAllStepContexts() {
    * 

To complete a concrete subclass, implement {@link #timerInternals} and * {@link #stateInternals}. */ - public abstract static class StepContext implements org.apache.beam.runners.core.StepContext { + public abstract static class BaseStepContext implements org.apache.beam.runners.core.StepContext { private final ExecutionContext executionContext; private final String stepName; private final String transformName; - public StepContext(ExecutionContext executionContext, String stepName, String transformName) { + public BaseStepContext(ExecutionContext executionContext, String stepName, String transformName) { this.executionContext = executionContext; this.stepName = stepName; this.transformName = transformName; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index abefd1c488b7..3750e6c9f0f7 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -29,7 +29,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import org.apache.beam.runners.core.BaseExecutionContext.StepContext; +import org.apache.beam.runners.core.BaseExecutionContext.BaseStepContext; import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.sdk.coders.Coder; @@ -63,7 +63,8 @@ public class SimpleDoFnRunnerTest { @Rule public ExpectedException thrown = ExpectedException.none(); - @Mock StepContext mockStepContext; + @Mock + BaseStepContext mockStepContext; @Mock TimerInternals mockTimerInternals; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java index 5172f433135a..a335c3a5daa4 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java @@ -24,7 +24,7 @@ import com.google.common.base.MoreObjects; import java.util.Collections; -import org.apache.beam.runners.core.BaseExecutionContext.StepContext; +import org.apache.beam.runners.core.BaseExecutionContext.BaseStepContext; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VarIntCoder; @@ -69,7 +69,8 @@ public class StatefulDoFnRunnerTest { private static final IntervalWindow WINDOW_2 = new IntervalWindow(new Instant(10), new Instant(20)); - @Mock StepContext mockStepContext; + @Mock + BaseStepContext mockStepContext; private InMemoryStateInternals stateInternals; private InMemoryTimerInternals timerInternals; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index 107f39ad4f74..6d2d02ab4fb2 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -57,7 +57,7 @@ protected DirectStepContext createStepContext(String stepName, String transformN * Step Context for the {@link DirectRunner}. */ public class DirectStepContext - extends BaseExecutionContext.StepContext { + extends BaseStepContext { private CopyOnAccessInMemoryStateInternals stateInternals; private DirectTimerInternals timerInternals; From 59322d51e80e7480710a296f51a4cb65303f5e06 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 15:35:46 -0700 Subject: [PATCH 041/159] Move BaseStepContext to the top level --- .../runners/core/BaseExecutionContext.java | 46 ------------- .../beam/runners/core/BaseStepContext.java | 66 +++++++++++++++++++ .../runners/core/SimpleDoFnRunnerTest.java | 1 - .../runners/core/StatefulDoFnRunnerTest.java | 1 - .../direct/DirectExecutionContext.java | 1 + 5 files changed, 67 insertions(+), 48 deletions(-) create mode 100644 runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java index 5667250b3288..877fa0a4f105 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java @@ -17,15 +17,10 @@ */ package org.apache.beam.runners.core; -import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.LinkedHashMap; import java.util.Map; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; /** * Base class for implementations of {@link ExecutionContext}. @@ -104,45 +99,4 @@ public Collection getAllStepContexts() { return Collections.unmodifiableCollection(cachedStepContexts.values()); } - /** - * Base class for implementations of {@link org.apache.beam.runners.core.StepContext}. - * - *

To complete a concrete subclass, implement {@link #timerInternals} and - * {@link #stateInternals}. - */ - public abstract static class BaseStepContext implements org.apache.beam.runners.core.StepContext { - private final ExecutionContext executionContext; - private final String stepName; - private final String transformName; - - public BaseStepContext(ExecutionContext executionContext, String stepName, String transformName) { - this.executionContext = executionContext; - this.stepName = stepName; - this.transformName = transformName; - } - - @Override - public String getStepName() { - return stepName; - } - - @Override - public String getTransformName() { - return transformName; - } - - @Override - public void writePCollectionViewData( - TupleTag tag, - Iterable> data, Coder>> dataCoder, - W window, Coder windowCoder) throws IOException { - throw new UnsupportedOperationException("Not implemented."); - } - - @Override - public abstract StateInternals stateInternals(); - - @Override - public abstract TimerInternals timerInternals(); - } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java new file mode 100644 index 000000000000..f0436acefdc5 --- /dev/null +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core; + +import java.io.IOException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TupleTag; + +/** + * Base class for implementations of {@link StepContext}. + * + *

To complete a concrete subclass, implement {@link #timerInternals} and + * {@link #stateInternals}. + */ +public abstract class BaseStepContext implements StepContext { + private final ExecutionContext executionContext; + private final String stepName; + private final String transformName; + + public BaseStepContext(ExecutionContext executionContext, String stepName, String transformName) { + this.executionContext = executionContext; + this.stepName = stepName; + this.transformName = transformName; + } + + @Override + public String getStepName() { + return stepName; + } + + @Override + public String getTransformName() { + return transformName; + } + + @Override + public void writePCollectionViewData( + TupleTag tag, + Iterable> data, Coder>> dataCoder, + W window, Coder windowCoder) throws IOException { + throw new UnsupportedOperationException("Not implemented."); + } + + @Override + public abstract StateInternals stateInternals(); + + @Override + public abstract TimerInternals timerInternals(); +} diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index 3750e6c9f0f7..59e5857c3aa1 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import org.apache.beam.runners.core.BaseExecutionContext.BaseStepContext; import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.sdk.coders.Coder; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java index a335c3a5daa4..62a657882cc6 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java @@ -24,7 +24,6 @@ import com.google.common.base.MoreObjects; import java.util.Collections; -import org.apache.beam.runners.core.BaseExecutionContext.BaseStepContext; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VarIntCoder; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index 6d2d02ab4fb2..e5b88e5ac8ee 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.direct; import org.apache.beam.runners.core.BaseExecutionContext; +import org.apache.beam.runners.core.BaseStepContext; import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; From 248c808a6603dc2c28a0b55296e0d596b8903a08 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 15:36:41 -0700 Subject: [PATCH 042/159] Remove extraneous ExecutionContext parameter to BaseStepContext --- .../java/org/apache/beam/runners/core/BaseStepContext.java | 4 +--- .../apache/beam/runners/direct/DirectExecutionContext.java | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java index f0436acefdc5..014fe0def371 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java @@ -30,12 +30,10 @@ * {@link #stateInternals}. */ public abstract class BaseStepContext implements StepContext { - private final ExecutionContext executionContext; private final String stepName; private final String transformName; - public BaseStepContext(ExecutionContext executionContext, String stepName, String transformName) { - this.executionContext = executionContext; + public BaseStepContext(String stepName, String transformName) { this.stepName = stepName; this.transformName = transformName; } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index e5b88e5ac8ee..d676f247298c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -64,7 +64,7 @@ public class DirectStepContext public DirectStepContext( ExecutionContext executionContext, String stepName, String transformName) { - super(executionContext, stepName, transformName); + super(stepName, transformName); } @Override From 5ac24e0a89b95feafccbe381bdde9c11fdf82a88 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 15:44:17 -0700 Subject: [PATCH 043/159] Implement StepContext directly in the DirectRunner --- .../direct/DirectExecutionContext.java | 33 ++++++++++++++++--- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index d676f247298c..2a75ef51a4ba 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -17,13 +17,18 @@ */ package org.apache.beam.runners.direct; +import java.io.IOException; import org.apache.beam.runners.core.BaseExecutionContext; -import org.apache.beam.runners.core.BaseStepContext; import org.apache.beam.runners.core.ExecutionContext; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TupleTag; /** * Execution Context for the {@link DirectRunner}. @@ -57,14 +62,16 @@ protected DirectStepContext createStepContext(String stepName, String transformN /** * Step Context for the {@link DirectRunner}. */ - public class DirectStepContext - extends BaseStepContext { + public class DirectStepContext implements StepContext { private CopyOnAccessInMemoryStateInternals stateInternals; private DirectTimerInternals timerInternals; + private final String stepName; + private final String transformName; public DirectStepContext( ExecutionContext executionContext, String stepName, String transformName) { - super(stepName, transformName); + this.stepName = stepName; + this.transformName = transformName; } @Override @@ -95,6 +102,24 @@ public CopyOnAccessInMemoryStateInternals commitState() { return null; } + @Override + public String getStepName() { + return stepName; + } + + @Override + public String getTransformName() { + return transformName; + } + + @Override + public void writePCollectionViewData( + TupleTag tag, + Iterable> data, Coder>> dataCoder, + W window, Coder windowCoder) throws IOException { + throw new UnsupportedOperationException("Not implemented."); + } + /** * Gets the timer update of the {@link TimerInternals} of this {@link DirectStepContext}, * which is empty if the {@link TimerInternals} were never accessed. From 32c6cb160f42e401f3e170cc8ed18d76c627d3e4 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 16:26:00 -0700 Subject: [PATCH 044/159] Remove writePCollectionViewData from the Beam codebase --- .../apex/translation/utils/NoOpStepContext.java | 13 ------------- .../beam/runners/core/BaseStepContext.java | 14 -------------- .../apache/beam/runners/core/StepContext.java | 17 ----------------- .../runners/direct/DirectExecutionContext.java | 13 ------------- .../functions/FlinkNoOpStepContext.java | 16 +--------------- .../wrappers/streaming/DoFnOperator.java | 11 ----------- .../spark/translation/SparkProcessContext.java | 14 +------------- .../beam/fn/harness/fake/FakeStepContext.java | 16 +--------------- 8 files changed, 3 insertions(+), 111 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java index 51e843b56652..820b1897b5db 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java @@ -17,15 +17,10 @@ */ package org.apache.beam.runners.apex.translation.utils; -import java.io.IOException; import java.io.Serializable; import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; /** * Serializable {@link StepContext} that does nothing. @@ -43,14 +38,6 @@ public String getTransformName() { return null; } - @Override - public void writePCollectionViewData(TupleTag tag, - Iterable> data, - Coder>> dataCoder, W window, Coder windowCoder) throws - IOException { - - } - @Override public StateInternals stateInternals() { return null; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java index 014fe0def371..e639c46c9f62 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java @@ -17,12 +17,6 @@ */ package org.apache.beam.runners.core; -import java.io.IOException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; - /** * Base class for implementations of {@link StepContext}. * @@ -48,14 +42,6 @@ public String getTransformName() { return transformName; } - @Override - public void writePCollectionViewData( - TupleTag tag, - Iterable> data, Coder>> dataCoder, - W window, Coder windowCoder) throws IOException { - throw new UnsupportedOperationException("Not implemented."); - } - @Override public abstract StateInternals stateInternals(); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java index fd2575d5f74b..62a81f154eca 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java @@ -17,12 +17,6 @@ */ package org.apache.beam.runners.core; -import java.io.IOException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; - /** * Per-step, per-key context used for retrieving state. */ @@ -38,17 +32,6 @@ public interface StepContext { */ String getTransformName(); - /** - * Writes the given {@code PCollectionView} data to a globally accessible location. - */ - void writePCollectionViewData( - TupleTag tag, - Iterable> data, - Coder>> dataCoder, - W window, - Coder windowCoder) - throws IOException; - StateInternals stateInternals(); TimerInternals timerInternals(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index 2a75ef51a4ba..39174d630954 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.direct; -import java.io.IOException; import org.apache.beam.runners.core.BaseExecutionContext; import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.StepContext; @@ -25,10 +24,6 @@ import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; /** * Execution Context for the {@link DirectRunner}. @@ -112,14 +107,6 @@ public String getTransformName() { return transformName; } - @Override - public void writePCollectionViewData( - TupleTag tag, - Iterable> data, Coder>> dataCoder, - W window, Coder windowCoder) throws IOException { - throw new UnsupportedOperationException("Not implemented."); - } - /** * Gets the timer update of the {@link TimerInternals} of this {@link DirectStepContext}, * which is empty if the {@link TimerInternals} were never accessed. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java index d999494eee79..1ff322ee0ba5 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java @@ -17,14 +17,9 @@ */ package org.apache.beam.runners.flink.translation.functions; -import java.io.IOException; -import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; /** * A {@link StepContext} for Flink Batch Runner execution. @@ -41,15 +36,6 @@ public String getTransformName() { return null; } - @Override - public void writePCollectionViewData( - TupleTag tag, - Iterable> data, - Coder>> dataCoder, - W window, - Coder windowCoder) throws IOException { - } - @Override public StateInternals stateInternals() { return null; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 2bb9c2003898..4f8998e9df8c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -23,7 +23,6 @@ import com.google.common.collect.Iterables; import java.io.DataInputStream; import java.io.DataOutputStream; -import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; @@ -687,16 +686,6 @@ public String getTransformName() { return null; } - @Override - public void writePCollectionViewData( - TupleTag tag, - Iterable> data, - Coder>> dataCoder, - W window, - Coder windowCoder) throws IOException { - throw new UnsupportedOperationException("Writing side-input data is not supported."); - } - @Override public StateInternals stateInternals() { return stateInternals; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java index 31e616cbf0f4..e693143ade16 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java @@ -20,19 +20,15 @@ import com.google.common.collect.AbstractIterator; import com.google.common.collect.Lists; -import java.io.IOException; import java.util.Iterator; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners.OutputManager; -import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; /** @@ -109,14 +105,6 @@ public String getTransformName() { return null; } - @Override - public void writePCollectionViewData( - TupleTag tag, - Iterable> data, - Coder>> dataCoder, - W window, - Coder windowCoder) throws IOException { } - @Override public StateInternals stateInternals() { return null; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java index 750c167d8993..3f6a2daed61c 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java @@ -18,14 +18,9 @@ package org.apache.beam.fn.harness.fake; -import java.io.IOException; -import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; /** * A fake {@link StepContext} factory that performs no-ops. @@ -41,15 +36,6 @@ public String getTransformName() { return "TODO"; } - @Override - public void writePCollectionViewData( - TupleTag tag, - Iterable> data, - Coder>> dataCoder, - W window, - Coder windowCoder) throws IOException { - } - @Override public StateInternals stateInternals() { throw new UnsupportedOperationException(); From 0be3cf3462c19f0b007b2329c95ea4865d22cad5 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 16:50:41 -0700 Subject: [PATCH 045/159] Inline and delete BaseExecutionContext --- .../runners/core/BaseExecutionContext.java | 102 ------------------ .../direct/DirectExecutionContext.java | 39 +++++-- 2 files changed, 32 insertions(+), 109 deletions(-) delete mode 100644 runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java deleted file mode 100644 index 877fa0a4f105..000000000000 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseExecutionContext.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.core; - -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.Map; - -/** - * Base class for implementations of {@link ExecutionContext}. - * - *

A concrete subclass should implement {@link #createStepContext} to create the appropriate - * {@link BaseStepContext} implementation. Any {@code StepContext} created will - * be cached for the lifetime of this {@link ExecutionContext}. - * - *

BaseExecutionContext is generic to allow implementing subclasses to return a concrete subclass - * of {@link BaseStepContext} from {@link #getOrCreateStepContext(String, String)} and - * {@link #getAllStepContexts()} without forcing each subclass to override the method, e.g. - *

{@code
- * {@literal @}Override
- * StreamingModeExecutionContext.StepContext getOrCreateStepContext(...) {
- *   return (StreamingModeExecutionContext.StepContext) super.getOrCreateStepContext(...);
- * }
- * }
- * - *

When a subclass of {@code BaseExecutionContext} has been downcast, the return types of - * {@link #createStepContext(String, String)}, - * {@link #getOrCreateStepContext(String, String)}, and {@link #getAllStepContexts()} - * will be appropriately specialized. - */ -public abstract class BaseExecutionContext - implements ExecutionContext { - - private Map cachedStepContexts = new LinkedHashMap<>(); - - /** - * Implementations should override this to create the specific type - * of {@link BaseStepContext} they need. - */ - protected abstract T createStepContext(String stepName, String transformName); - - /** - * Returns the {@link BaseStepContext} associated with the given step. - */ - @Override - public T getOrCreateStepContext(String stepName, String transformName) { - final String finalStepName = stepName; - final String finalTransformName = transformName; - return getOrCreateStepContext( - stepName, - new CreateStepContextFunction() { - @Override - public T create() { - return createStepContext(finalStepName, finalTransformName); - } - }); - } - - /** - * Factory method interface to create an execution context if none exists during - * {@link #getOrCreateStepContext(String, CreateStepContextFunction)}. - */ - protected interface CreateStepContextFunction { - T create(); - } - - protected final T getOrCreateStepContext(String stepName, - CreateStepContextFunction createContextFunc) { - T context = cachedStepContexts.get(stepName); - if (context == null) { - context = createContextFunc.create(); - cachedStepContexts.put(stepName, context); - } - - return context; - } - - /** - * Returns a collection view of all of the {@link BaseStepContext}s. - */ - @Override - public Collection getAllStepContexts() { - return Collections.unmodifiableCollection(cachedStepContexts.values()); - } - -} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index 39174d630954..9b6866216a61 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -17,11 +17,14 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.runners.core.BaseExecutionContext; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Map; +import org.apache.beam.runners.core.BaseStepContext; import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; @@ -31,12 +34,12 @@ *

This implementation is not thread safe. A new {@link DirectExecutionContext} must be created * for each thread that requires it. */ -class DirectExecutionContext - extends BaseExecutionContext { +class DirectExecutionContext implements ExecutionContext { private final Clock clock; private final StructuralKey key; private final CopyOnAccessInMemoryStateInternals existingState; private final TransformWatermarks watermarks; + private Map cachedStepContexts = new LinkedHashMap<>(); public DirectExecutionContext( Clock clock, @@ -49,9 +52,31 @@ public DirectExecutionContext( this.watermarks = watermarks; } + private DirectStepContext createStepContext(String stepName, String transformName) { + return new DirectStepContext(stepName, transformName); + } + + /** + * Returns the {@link BaseStepContext} associated with the given step. + */ + @Override + public DirectStepContext getOrCreateStepContext(String stepName, String transformName) { + final String finalStepName = stepName; + final String finalTransformName = transformName; + DirectStepContext context = cachedStepContexts.get(stepName); + if (context == null) { + context = createStepContext(finalStepName, finalTransformName); + cachedStepContexts.put(stepName, context); + } + return context; + } + + /** + * Returns a collection view of all of the {@link BaseStepContext}s. + */ @Override - protected DirectStepContext createStepContext(String stepName, String transformName) { - return new DirectStepContext(this, stepName, transformName); + public Collection getAllStepContexts() { + return Collections.unmodifiableCollection(cachedStepContexts.values()); } /** @@ -64,7 +89,7 @@ public class DirectStepContext implements StepContext { private final String transformName; public DirectStepContext( - ExecutionContext executionContext, String stepName, String transformName) { + String stepName, String transformName) { this.stepName = stepName; this.transformName = transformName; } From 97c230af62151fdbe06ac622282d69c74db30b2f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 17:33:38 -0700 Subject: [PATCH 046/159] Delete unused ExecutionContext --- .../beam/runners/core/ExecutionContext.java | 36 ------------------- .../direct/DirectExecutionContext.java | 3 +- .../runners/direct/EvaluationContext.java | 29 +++++++-------- 3 files changed, 14 insertions(+), 54 deletions(-) delete mode 100644 runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java deleted file mode 100644 index eac3599d9757..000000000000 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ExecutionContext.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.core; - -import java.util.Collection; - -/** - * Context for the current execution. This is guaranteed to exist during processing, - * but does not necessarily persist between different batches of work. - */ -public interface ExecutionContext { - /** - * Returns the {@link StepContext} associated with the given step. - */ - StepContext getOrCreateStepContext(String stepName, String transformName); - - /** - * Returns a collection view of all of the {@link StepContext}s. - */ - Collection getAllStepContexts(); -} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index 9b6866216a61..05dbebc053c7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -22,7 +22,6 @@ import java.util.LinkedHashMap; import java.util.Map; import org.apache.beam.runners.core.BaseStepContext; -import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; @@ -34,7 +33,7 @@ *

This implementation is not thread safe. A new {@link DirectExecutionContext} must be created * for each thread that requires it. */ -class DirectExecutionContext implements ExecutionContext { +class DirectExecutionContext { private final Clock clock; private final StructuralKey key; private final CopyOnAccessInMemoryStateInternals existingState; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index c62711996648..88ce85a7cd22 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -31,7 +31,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import javax.annotation.Nullable; -import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.ReadyCheckingSideInputReader; import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.runners.core.TimerInternals.TimerData; @@ -52,22 +51,20 @@ import org.joda.time.Instant; /** - * The evaluation context for a specific pipeline being executed by the - * {@link DirectRunner}. Contains state shared within the execution across all - * transforms. + * The evaluation context for a specific pipeline being executed by the {@link DirectRunner}. + * Contains state shared within the execution across all transforms. * - *

{@link EvaluationContext} contains shared state for an execution of the - * {@link DirectRunner} that can be used while evaluating a {@link PTransform}. This - * consists of views into underlying state and watermark implementations, access to read and write - * {@link PCollectionView PCollectionViews}, and managing the - * {@link ExecutionContext ExecutionContexts}. This includes executing callbacks asynchronously when - * state changes to the appropriate point (e.g. when a {@link PCollectionView} is requested and - * known to be empty). + *

{@link EvaluationContext} contains shared state for an execution of the {@link DirectRunner} + * that can be used while evaluating a {@link PTransform}. This consists of views into underlying + * state and watermark implementations, access to read and write {@link PCollectionView + * PCollectionViews}, and managing the {@link DirectExecutionContext ExecutionContexts}. This + * includes executing callbacks asynchronously when state changes to the appropriate point (e.g. + * when a {@link PCollectionView} is requested and known to be empty). * - *

{@link EvaluationContext} also handles results by committing finalizing bundles based - * on the current global state and updating the global state appropriately. This includes updating - * the per-{@link StepAndKey} state, updating global watermarks, and executing any callbacks that - * can be executed. + *

{@link EvaluationContext} also handles results by committing finalizing bundles based on the + * current global state and updating the global state appropriately. This includes updating the + * per-{@link StepAndKey} state, updating global watermarks, and executing any callbacks that can be + * executed. */ class EvaluationContext { /** @@ -312,7 +309,7 @@ public DirectOptions getPipelineOptions() { } /** - * Get an {@link ExecutionContext} for the provided {@link AppliedPTransform} and key. + * Get a {@link DirectExecutionContext} for the provided {@link AppliedPTransform} and key. */ public DirectExecutionContext getExecutionContext( AppliedPTransform application, StructuralKey key) { From acce24ce1388b7953fbb9d87da5bb2271286c58a Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 17:42:10 -0700 Subject: [PATCH 047/159] Remove unused StepContext name methods --- .../apex/translation/utils/NoOpStepContext.java | 10 ---------- .../org/apache/beam/runners/core/BaseStepContext.java | 10 ---------- .../java/org/apache/beam/runners/core/StepContext.java | 10 ---------- .../beam/runners/direct/DirectExecutionContext.java | 10 ---------- .../translation/functions/FlinkNoOpStepContext.java | 10 ---------- .../translation/wrappers/streaming/DoFnOperator.java | 10 ---------- .../runners/spark/translation/SparkProcessContext.java | 9 --------- .../apache/beam/fn/harness/fake/FakeStepContext.java | 9 --------- 8 files changed, 78 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java index 820b1897b5db..b49e4da27bcd 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/NoOpStepContext.java @@ -28,16 +28,6 @@ public class NoOpStepContext implements StepContext, Serializable { private static final long serialVersionUID = 1L; - @Override - public String getStepName() { - return null; - } - - @Override - public String getTransformName() { - return null; - } - @Override public StateInternals stateInternals() { return null; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java index e639c46c9f62..4abd4d228160 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java @@ -32,16 +32,6 @@ public BaseStepContext(String stepName, String transformName) { this.transformName = transformName; } - @Override - public String getStepName() { - return stepName; - } - - @Override - public String getTransformName() { - return transformName; - } - @Override public abstract StateInternals stateInternals(); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java index 62a81f154eca..60fc40291452 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java @@ -22,16 +22,6 @@ */ public interface StepContext { - /** - * The name of the step. - */ - String getStepName(); - - /** - * The name of the transform for the step. - */ - String getTransformName(); - StateInternals stateInternals(); TimerInternals timerInternals(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index 05dbebc053c7..651af8f4f818 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -121,16 +121,6 @@ public CopyOnAccessInMemoryStateInternals commitState() { return null; } - @Override - public String getStepName() { - return stepName; - } - - @Override - public String getTransformName() { - return transformName; - } - /** * Gets the timer update of the {@link TimerInternals} of this {@link DirectStepContext}, * which is empty if the {@link TimerInternals} were never accessed. diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java index 1ff322ee0ba5..9c7b63677ba3 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoOpStepContext.java @@ -26,16 +26,6 @@ */ public class FlinkNoOpStepContext implements StepContext { - @Override - public String getStepName() { - return null; - } - - @Override - public String getTransformName() { - return null; - } - @Override public StateInternals stateInternals() { return null; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 4f8998e9df8c..d2ab7e1cabbd 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -676,16 +676,6 @@ public void output(TupleTag tag, WindowedValue value) { */ protected class StepContext implements org.apache.beam.runners.core.StepContext { - @Override - public String getStepName() { - return null; - } - - @Override - public String getTransformName() { - return null; - } - @Override public StateInternals stateInternals() { return stateInternals; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java index e693143ade16..f4ab7d9b02cb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java @@ -95,15 +95,6 @@ interface SparkOutputManager extends OutputManager, Iterable { } static class NoOpStepContext implements StepContext { - @Override - public String getStepName() { - return null; - } - - @Override - public String getTransformName() { - return null; - } @Override public StateInternals stateInternals() { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java index 3f6a2daed61c..bdf138b31523 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java @@ -26,15 +26,6 @@ * A fake {@link StepContext} factory that performs no-ops. */ public class FakeStepContext implements StepContext { - @Override - public String getStepName() { - return "TODO"; - } - - @Override - public String getTransformName() { - return "TODO"; - } @Override public StateInternals stateInternals() { From dc585510e6b4def3a0442114d77e96f2b5d4880f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 17:46:12 -0700 Subject: [PATCH 048/159] Delete unused remnants in DirectExecutionContext --- .../beam/runners/direct/DirectExecutionContext.java | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index 651af8f4f818..845256539eb9 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -56,9 +56,8 @@ private DirectStepContext createStepContext(String stepName, String transformNam } /** - * Returns the {@link BaseStepContext} associated with the given step. + * Returns the {@link StepContext} associated with the given step. */ - @Override public DirectStepContext getOrCreateStepContext(String stepName, String transformName) { final String finalStepName = stepName; final String finalTransformName = transformName; @@ -70,14 +69,6 @@ public DirectStepContext getOrCreateStepContext(String stepName, String transfor return context; } - /** - * Returns a collection view of all of the {@link BaseStepContext}s. - */ - @Override - public Collection getAllStepContexts() { - return Collections.unmodifiableCollection(cachedStepContexts.values()); - } - /** * Step Context for the {@link DirectRunner}. */ From 62115b29a7f27a1a74b7c870d4277655adb3dfbf Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 17:46:58 -0700 Subject: [PATCH 049/159] Delete unused BaseStepContext --- .../beam/runners/core/BaseStepContext.java | 40 ------------------- .../runners/core/SimpleDoFnRunnerTest.java | 2 +- .../runners/core/StatefulDoFnRunnerTest.java | 2 +- .../direct/DirectExecutionContext.java | 3 -- 4 files changed, 2 insertions(+), 45 deletions(-) delete mode 100644 runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java deleted file mode 100644 index 4abd4d228160..000000000000 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/BaseStepContext.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.core; - -/** - * Base class for implementations of {@link StepContext}. - * - *

To complete a concrete subclass, implement {@link #timerInternals} and - * {@link #stateInternals}. - */ -public abstract class BaseStepContext implements StepContext { - private final String stepName; - private final String transformName; - - public BaseStepContext(String stepName, String transformName) { - this.stepName = stepName; - this.transformName = transformName; - } - - @Override - public abstract StateInternals stateInternals(); - - @Override - public abstract TimerInternals timerInternals(); -} diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index 59e5857c3aa1..f331b65abcb8 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -63,7 +63,7 @@ public class SimpleDoFnRunnerTest { @Rule public ExpectedException thrown = ExpectedException.none(); @Mock - BaseStepContext mockStepContext; + StepContext mockStepContext; @Mock TimerInternals mockTimerInternals; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java index 62a657882cc6..4f155dca4c3a 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java @@ -69,7 +69,7 @@ public class StatefulDoFnRunnerTest { new IntervalWindow(new Instant(10), new Instant(20)); @Mock - BaseStepContext mockStepContext; + StepContext mockStepContext; private InMemoryStateInternals stateInternals; private InMemoryTimerInternals timerInternals; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index 845256539eb9..cca57193fbd2 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -17,11 +17,8 @@ */ package org.apache.beam.runners.direct; -import java.util.Collection; -import java.util.Collections; import java.util.LinkedHashMap; import java.util.Map; -import org.apache.beam.runners.core.BaseStepContext; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; From 0dc0334a0c1350c1693019f104dac911a618c9c8 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 17:49:01 -0700 Subject: [PATCH 050/159] Shorten excessive name in DirectExecutionContext --- .../runners/direct/DirectExecutionContext.java | 2 +- .../GroupAlsoByWindowEvaluatorFactory.java | 2 +- .../runners/direct/ParDoEvaluatorFactory.java | 2 +- ...plittableProcessElementsEvaluatorFactory.java | 2 +- .../direct/StatefulParDoEvaluatorFactory.java | 2 +- .../runners/direct/EvaluationContextTest.java | 16 ++++++++-------- .../beam/runners/direct/ParDoEvaluatorTest.java | 2 +- .../StatefulParDoEvaluatorFactoryTest.java | 4 ++-- 8 files changed, 16 insertions(+), 16 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index cca57193fbd2..11c1b86a559c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -55,7 +55,7 @@ private DirectStepContext createStepContext(String stepName, String transformNam /** * Returns the {@link StepContext} associated with the given step. */ - public DirectStepContext getOrCreateStepContext(String stepName, String transformName) { + public DirectStepContext getStepContext(String stepName, String transformName) { final String finalStepName = stepName; final String finalTransformName = transformName; DirectStepContext context = cachedStepContexts.get(stepName); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 78ef7fe5d6da..49b75129b45c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -129,7 +129,7 @@ public GroupAlsoByWindowEvaluator( structuralKey = inputBundle.getKey(); stepContext = evaluationContext .getExecutionContext(application, inputBundle.getKey()) - .getOrCreateStepContext( + .getStepContext( evaluationContext.getStepName(application), application.getTransform().getName()); windowingStrategy = (WindowingStrategy) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index 74470bfb8b8d..12c6751f8e7e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -112,7 +112,7 @@ DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( DirectStepContext stepContext = evaluationContext .getExecutionContext(application, inputBundleKey) - .getOrCreateStepContext(stepName, stepName); + .getStepContext(stepName, stepName); DoFnLifecycleManager fnManager = fnClones.getUnchecked(doFn); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index dc85d87bc93f..13d9345094fe 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -109,7 +109,7 @@ public void cleanup() throws Exception { final DirectExecutionContext.DirectStepContext stepContext = evaluationContext .getExecutionContext(application, inputBundle.getKey()) - .getOrCreateStepContext(stepName, stepName); + .getStepContext(stepName, stepName); final ParDoEvaluator>> parDoEvaluator = diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 985c3be4e9e9..70d0cf5bf8f1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -163,7 +163,7 @@ public Runnable load( evaluationContext .getExecutionContext( transformOutputWindow.getTransform(), transformOutputWindow.getKey()) - .getOrCreateStepContext(stepName, stepName); + .getStepContext(stepName, stepName); final StateNamespace namespace = StateNamespaces.window( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index 72b1bbcba188..0e2be8d4d5b7 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -160,7 +160,7 @@ public void getExecutionContextSameStepSameKeyState() { StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); - DirectStepContext stepContext = fooContext.getOrCreateStepContext("s1", "s1"); + DirectStepContext stepContext = fooContext.getStepContext("s1", "s1"); stepContext.stateInternals().state(StateNamespaces.global(), intBag).add(1); context.handleResult( @@ -177,7 +177,7 @@ public void getExecutionContextSameStepSameKeyState() { StructuralKey.of("foo", StringUtf8Coder.of())); assertThat( secondFooContext - .getOrCreateStepContext("s1", "s1") + .getStepContext("s1", "s1") .stateInternals() .state(StateNamespaces.global(), intBag) .read(), @@ -194,7 +194,7 @@ public void getExecutionContextDifferentKeysIndependentState() { StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); fooContext - .getOrCreateStepContext("s1", "s1") + .getStepContext("s1", "s1") .stateInternals() .state(StateNamespaces.global(), intBag) .add(1); @@ -205,7 +205,7 @@ public void getExecutionContextDifferentKeysIndependentState() { assertThat(barContext, not(equalTo(fooContext))); assertThat( barContext - .getOrCreateStepContext("s1", "s1") + .getStepContext("s1", "s1") .stateInternals() .state(StateNamespaces.global(), intBag) .read(), @@ -221,7 +221,7 @@ public void getExecutionContextDifferentStepsIndependentState() { StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); fooContext - .getOrCreateStepContext("s1", "s1") + .getStepContext("s1", "s1") .stateInternals() .state(StateNamespaces.global(), intBag) .add(1); @@ -230,7 +230,7 @@ public void getExecutionContextDifferentStepsIndependentState() { context.getExecutionContext(downstreamProducer, myKey); assertThat( barContext - .getOrCreateStepContext("s1", "s1") + .getStepContext("s1", "s1") .stateInternals() .state(StateNamespaces.global(), intBag) .read(), @@ -246,7 +246,7 @@ public void handleResultStoresState() { StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); CopyOnAccessInMemoryStateInternals state = - fooContext.getOrCreateStepContext("s1", "s1").stateInternals(); + fooContext.getStepContext("s1", "s1").stateInternals(); BagState bag = state.state(StateNamespaces.global(), intBag); bag.add(1); bag.add(2); @@ -266,7 +266,7 @@ public void handleResultStoresState() { context.getExecutionContext(downstreamProducer, myKey); CopyOnAccessInMemoryStateInternals afterResultState = - afterResultContext.getOrCreateStepContext("s1", "s1").stateInternals(); + afterResultContext.getStepContext("s1", "s1").stateInternals(); assertThat(afterResultState.state(StateNamespaces.global(), intBag).read(), contains(1, 2, 4)); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index 286e44d1be04..22b3b7e49096 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -140,7 +140,7 @@ private ParDoEvaluator createEvaluator( DirectExecutionContext executionContext = mock(DirectExecutionContext.class); DirectStepContext stepContext = mock(DirectStepContext.class); when( - executionContext.getOrCreateStepContext( + executionContext.getStepContext( Mockito.any(String.class), Mockito.any(String.class))) .thenReturn(stepContext); when(stepContext.getTimerUpdate()).thenReturn(TimerUpdate.empty()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index eb54d5c818dc..b233c1bfe868 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -153,7 +153,7 @@ public void process(ProcessContext c) {} when(mockEvaluationContext.getExecutionContext( eq(producingTransform), Mockito.any())) .thenReturn(mockExecutionContext); - when(mockExecutionContext.getOrCreateStepContext(anyString(), anyString())) + when(mockExecutionContext.getStepContext(anyString(), anyString())) .thenReturn(mockStepContext); IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(9)); @@ -269,7 +269,7 @@ public void process(ProcessContext c) {} when(mockEvaluationContext.getExecutionContext( eq(producingTransform), Mockito.any())) .thenReturn(mockExecutionContext); - when(mockExecutionContext.getOrCreateStepContext(anyString(), anyString())) + when(mockExecutionContext.getStepContext(anyString(), anyString())) .thenReturn(mockStepContext); when(mockEvaluationContext.createBundle(Matchers.>any())) .thenReturn(mockUncommittedBundle); From d425b2792f754ed6150f7b47eddf743286a45401 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 11:10:42 -0700 Subject: [PATCH 051/159] Revise StepContext javadoc --- .../java/org/apache/beam/runners/core/StepContext.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java index 60fc40291452..4d66d668c7ff 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StepContext.java @@ -18,7 +18,12 @@ package org.apache.beam.runners.core; /** - * Per-step, per-key context used for retrieving state. + * The context in which a specific step is executing, including access to state and timers. + * + *

This interface exists as the API between a runner and the support code, but is not user + * facing. + * + *

These will often be scoped to a particular step and key, though it is not required. */ public interface StepContext { From b32a1c350398a91b1b1552d5257dab6ab7d1da3a Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 11:13:19 -0700 Subject: [PATCH 052/159] Remove unused pieces of DirectStepContext --- .../runners/direct/DirectExecutionContext.java | 18 +++++------------- .../GroupAlsoByWindowEvaluatorFactory.java | 2 +- .../runners/direct/ParDoEvaluatorFactory.java | 2 +- ...ittableProcessElementsEvaluatorFactory.java | 2 +- .../direct/StatefulParDoEvaluatorFactory.java | 2 +- .../runners/direct/EvaluationContextTest.java | 16 ++++++++-------- .../runners/direct/ParDoEvaluatorTest.java | 2 +- .../StatefulParDoEvaluatorFactoryTest.java | 4 ++-- 8 files changed, 20 insertions(+), 28 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java index 11c1b86a559c..e8ad8d75f351 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java @@ -48,19 +48,17 @@ public DirectExecutionContext( this.watermarks = watermarks; } - private DirectStepContext createStepContext(String stepName, String transformName) { - return new DirectStepContext(stepName, transformName); + private DirectStepContext createStepContext() { + return new DirectStepContext(); } /** * Returns the {@link StepContext} associated with the given step. */ - public DirectStepContext getStepContext(String stepName, String transformName) { - final String finalStepName = stepName; - final String finalTransformName = transformName; + public DirectStepContext getStepContext(String stepName) { DirectStepContext context = cachedStepContexts.get(stepName); if (context == null) { - context = createStepContext(finalStepName, finalTransformName); + context = createStepContext(); cachedStepContexts.put(stepName, context); } return context; @@ -72,14 +70,8 @@ public DirectStepContext getStepContext(String stepName, String transformName) { public class DirectStepContext implements StepContext { private CopyOnAccessInMemoryStateInternals stateInternals; private DirectTimerInternals timerInternals; - private final String stepName; - private final String transformName; - public DirectStepContext( - String stepName, String transformName) { - this.stepName = stepName; - this.transformName = transformName; - } + public DirectStepContext() { } @Override public CopyOnAccessInMemoryStateInternals stateInternals() { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 49b75129b45c..1a588eedf895 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -130,7 +130,7 @@ public GroupAlsoByWindowEvaluator( stepContext = evaluationContext .getExecutionContext(application, inputBundle.getKey()) .getStepContext( - evaluationContext.getStepName(application), application.getTransform().getName()); + evaluationContext.getStepName(application)); windowingStrategy = (WindowingStrategy) application.getTransform().getInputWindowingStrategy(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index 12c6751f8e7e..8aa75cf1445e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -112,7 +112,7 @@ DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( DirectStepContext stepContext = evaluationContext .getExecutionContext(application, inputBundleKey) - .getStepContext(stepName, stepName); + .getStepContext(stepName); DoFnLifecycleManager fnManager = fnClones.getUnchecked(doFn); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index 13d9345094fe..b85f481c1489 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -109,7 +109,7 @@ public void cleanup() throws Exception { final DirectExecutionContext.DirectStepContext stepContext = evaluationContext .getExecutionContext(application, inputBundle.getKey()) - .getStepContext(stepName, stepName); + .getStepContext(stepName); final ParDoEvaluator>> parDoEvaluator = diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 70d0cf5bf8f1..506c84cec639 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -163,7 +163,7 @@ public Runnable load( evaluationContext .getExecutionContext( transformOutputWindow.getTransform(), transformOutputWindow.getKey()) - .getStepContext(stepName, stepName); + .getStepContext(stepName); final StateNamespace namespace = StateNamespaces.window( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index 0e2be8d4d5b7..80b04f80a06a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -160,7 +160,7 @@ public void getExecutionContextSameStepSameKeyState() { StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); - DirectStepContext stepContext = fooContext.getStepContext("s1", "s1"); + DirectStepContext stepContext = fooContext.getStepContext("s1"); stepContext.stateInternals().state(StateNamespaces.global(), intBag).add(1); context.handleResult( @@ -177,7 +177,7 @@ public void getExecutionContextSameStepSameKeyState() { StructuralKey.of("foo", StringUtf8Coder.of())); assertThat( secondFooContext - .getStepContext("s1", "s1") + .getStepContext("s1") .stateInternals() .state(StateNamespaces.global(), intBag) .read(), @@ -194,7 +194,7 @@ public void getExecutionContextDifferentKeysIndependentState() { StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); fooContext - .getStepContext("s1", "s1") + .getStepContext("s1") .stateInternals() .state(StateNamespaces.global(), intBag) .add(1); @@ -205,7 +205,7 @@ public void getExecutionContextDifferentKeysIndependentState() { assertThat(barContext, not(equalTo(fooContext))); assertThat( barContext - .getStepContext("s1", "s1") + .getStepContext("s1") .stateInternals() .state(StateNamespaces.global(), intBag) .read(), @@ -221,7 +221,7 @@ public void getExecutionContextDifferentStepsIndependentState() { StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); fooContext - .getStepContext("s1", "s1") + .getStepContext("s1") .stateInternals() .state(StateNamespaces.global(), intBag) .add(1); @@ -230,7 +230,7 @@ public void getExecutionContextDifferentStepsIndependentState() { context.getExecutionContext(downstreamProducer, myKey); assertThat( barContext - .getStepContext("s1", "s1") + .getStepContext("s1") .stateInternals() .state(StateNamespaces.global(), intBag) .read(), @@ -246,7 +246,7 @@ public void handleResultStoresState() { StateTag> intBag = StateTags.bag("myBag", VarIntCoder.of()); CopyOnAccessInMemoryStateInternals state = - fooContext.getStepContext("s1", "s1").stateInternals(); + fooContext.getStepContext("s1").stateInternals(); BagState bag = state.state(StateNamespaces.global(), intBag); bag.add(1); bag.add(2); @@ -266,7 +266,7 @@ public void handleResultStoresState() { context.getExecutionContext(downstreamProducer, myKey); CopyOnAccessInMemoryStateInternals afterResultState = - afterResultContext.getStepContext("s1", "s1").stateInternals(); + afterResultContext.getStepContext("s1").stateInternals(); assertThat(afterResultState.state(StateNamespaces.global(), intBag).read(), contains(1, 2, 4)); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index 22b3b7e49096..09a21ac524a5 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -141,7 +141,7 @@ private ParDoEvaluator createEvaluator( DirectStepContext stepContext = mock(DirectStepContext.class); when( executionContext.getStepContext( - Mockito.any(String.class), Mockito.any(String.class))) + Mockito.any(String.class))) .thenReturn(stepContext); when(stepContext.getTimerUpdate()).thenReturn(TimerUpdate.empty()); when( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index b233c1bfe868..9366b7c9ff8c 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -153,7 +153,7 @@ public void process(ProcessContext c) {} when(mockEvaluationContext.getExecutionContext( eq(producingTransform), Mockito.any())) .thenReturn(mockExecutionContext); - when(mockExecutionContext.getStepContext(anyString(), anyString())) + when(mockExecutionContext.getStepContext(anyString())) .thenReturn(mockStepContext); IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(9)); @@ -269,7 +269,7 @@ public void process(ProcessContext c) {} when(mockEvaluationContext.getExecutionContext( eq(producingTransform), Mockito.any())) .thenReturn(mockExecutionContext); - when(mockExecutionContext.getStepContext(anyString(), anyString())) + when(mockExecutionContext.getStepContext(anyString())) .thenReturn(mockStepContext); when(mockEvaluationContext.createBundle(Matchers.>any())) .thenReturn(mockUncommittedBundle); From 329e4bd3bfa3414b661c8253f612f48527c783c0 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Tue, 23 May 2017 10:42:54 -0700 Subject: [PATCH 053/159] Fix lint error in datastoreio_test --- sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py index 424e714c5737..94cac3e58dcc 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio_test.py @@ -15,7 +15,6 @@ # limitations under the License. # -import math import unittest from mock import MagicMock, call, patch From 2e86f4faf9cba1fc15969f3c1f3c3463a332f7ca Mon Sep 17 00:00:00 2001 From: Sourabh Bajaj Date: Tue, 23 May 2017 13:24:27 -0700 Subject: [PATCH 054/159] [BEAM-2338] Fix the limit counter in gcsio reads --- sdks/python/apache_beam/io/gcp/gcsio.py | 2 +- sdks/python/apache_beam/io/gcp/gcsio_test.py | 22 +++++++++++++++++++- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py index 7e21586381c2..d43c8ba0d972 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio.py +++ b/sdks/python/apache_beam/io/gcp/gcsio.py @@ -392,7 +392,7 @@ def size_of_files_in_glob(self, pattern, limit=None): if fnmatch.fnmatch(item.name, name_pattern): file_name = 'gs://%s/%s' % (item.bucket, item.name) file_sizes[file_name] = item.size - counter += 1 + counter += 1 if limit is not None and counter >= limit: break if counter % 10000 == 0: diff --git a/sdks/python/apache_beam/io/gcp/gcsio_test.py b/sdks/python/apache_beam/io/gcp/gcsio_test.py index 73d221376a5d..06a82272900e 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio_test.py +++ b/sdks/python/apache_beam/io/gcp/gcsio_test.py @@ -641,6 +641,7 @@ def test_glob(self): 'apple/fish/cat', 'apple/fish/cart', 'apple/fish/carl', + 'apple/fish/handle', 'apple/dish/bat', 'apple/dish/cat', 'apple/dish/carl', @@ -661,6 +662,7 @@ def test_glob(self): 'apple/fish/cat', 'apple/fish/cart', 'apple/fish/carl', + 'apple/fish/handle', 'apple/dish/bat', 'apple/dish/cat', 'apple/dish/carl', @@ -691,6 +693,12 @@ def test_glob(self): 'apple/fish/bambi', 'apple/fish/balloon', ]), + ('gs://gcsio-test/apple/f*/b*', [ + 'apple/fish/blubber', + 'apple/fish/blowfish', + 'apple/fish/bambi', + 'apple/fish/balloon', + ]), ('gs://gcsio-test/apple/dish/[cb]at', [ 'apple/dish/bat', 'apple/dish/cat', @@ -726,6 +734,7 @@ def test_size_of_files_in_glob(self): ('apple/dish/bat', 13), ('apple/dish/cat', 14), ('apple/dish/carl', 15), + ('apple/fish/handle', 16), ] for (object_name, size) in object_names: file_name = 'gs://%s/%s' % (bucket_name, object_name) @@ -739,7 +748,11 @@ def test_size_of_files_in_glob(self): ('gs://gcsio-test/apple/fish/car?', [ ('apple/fish/cart', 11), ('apple/fish/carl', 12), - ]) + ]), + ('gs://gcsio-test/*/f*/car?', [ + ('apple/fish/cart', 11), + ('apple/fish/carl', 12), + ]), ] for file_pattern, expected_object_names in test_cases: expected_file_sizes = {'gs://%s/%s' % (bucket_name, o): s @@ -747,6 +760,13 @@ def test_size_of_files_in_glob(self): self.assertEqual( self.gcs.size_of_files_in_glob(file_pattern), expected_file_sizes) + # Check if limits are followed correctly + limit = 1 + for file_pattern, expected_object_names in test_cases: + expected_num_items = min(len(expected_object_names), limit) + self.assertEqual( + len(self.gcs.glob(file_pattern, limit)), expected_num_items) + def test_size_of_files_in_glob_limited(self): bucket_name = 'gcsio-test' object_names = [ From 940819e2acccb8d77b88aaee821ee972aca02eb3 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 15:26:44 -0700 Subject: [PATCH 055/159] Rename PCollections to PCollectionTranslation This is to give a standard and obvious suffix for all our helper classes for translating to/from Runner API protos. --- ...lections.java => PCollectionTranslation.java} | 4 ++-- .../runners/core/construction/SdkComponents.java | 3 ++- ...Test.java => PCollectionTranslationTest.java} | 16 +++++++++------- 3 files changed, 13 insertions(+), 10 deletions(-) rename runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/{PCollections.java => PCollectionTranslation.java} (98%) rename runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/{PCollectionsTest.java => PCollectionTranslationTest.java} (92%) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollections.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollections.java rename to runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java index 0f2fcb7ca281..cad7b97c74b5 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollections.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java @@ -29,8 +29,8 @@ /** * Utility methods for translating {@link PCollection PCollections} to and from Runner API protos. */ -public class PCollections { - private PCollections() {} +public class PCollectionTranslation { + private PCollectionTranslation() {} public static RunnerApi.PCollection toProto(PCollection pCollection, SdkComponents components) throws IOException { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java index 5714fc510481..3d8d4cd3b94e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java @@ -176,7 +176,8 @@ String registerPCollection(PCollection pCollection) throws IOException { } String uniqueName = uniqify(pCollection.getName(), pCollectionIds.values()); pCollectionIds.put(pCollection, uniqueName); - componentsBuilder.putPcollections(uniqueName, PCollections.toProto(pCollection, this)); + componentsBuilder.putPcollections( + uniqueName, PCollectionTranslation.toProto(pCollection, this)); return uniqueName; } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java similarity index 92% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java rename to runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java index 9407a5a5cda2..3b942206b112 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java @@ -64,10 +64,10 @@ import org.junit.runners.Parameterized.Parameters; /** - * Tests for {@link PCollections}. + * Tests for {@link PCollectionTranslation}. */ @RunWith(Parameterized.class) -public class PCollectionsTest { +public class PCollectionTranslationTest { // Each spec activates tests of all subsets of its fields @Parameters(name = "{index}: {0}") public static Iterable> data() { @@ -91,7 +91,8 @@ public static Iterable> data() { pipeline .apply( "intsWithCustomCoder", - Create.of(1, 2).withCoder(new AutoValue_PCollectionsTest_CustomIntCoder())) + Create.of(1, 2) + .withCoder(new AutoValue_PCollectionTranslationTest_CustomIntCoder())) .apply( "into custom windows", Window.into(new CustomWindows()) @@ -113,12 +114,13 @@ public static Iterable> data() { @Test public void testEncodeDecodeCycle() throws Exception { SdkComponents sdkComponents = SdkComponents.create(); - RunnerApi.PCollection protoCollection = PCollections.toProto(testCollection, sdkComponents); + RunnerApi.PCollection protoCollection = PCollectionTranslation + .toProto(testCollection, sdkComponents); RunnerApi.Components protoComponents = sdkComponents.toComponents(); - Coder decodedCoder = PCollections.getCoder(protoCollection, protoComponents); + Coder decodedCoder = PCollectionTranslation.getCoder(protoCollection, protoComponents); WindowingStrategy decodedStrategy = - PCollections.getWindowingStrategy(protoCollection, protoComponents); - IsBounded decodedIsBounded = PCollections.isBounded(protoCollection); + PCollectionTranslation.getWindowingStrategy(protoCollection, protoComponents); + IsBounded decodedIsBounded = PCollectionTranslation.isBounded(protoCollection); assertThat(decodedCoder, Matchers.>equalTo(testCollection.getCoder())); assertThat( From 9b6728e24748791b7181b20183df3ada31f45682 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 15:28:08 -0700 Subject: [PATCH 056/159] Rename PTransforms to PTransformTranslation --- .../{PTransforms.java => PTransformTranslation.java} | 4 ++-- .../apache/beam/runners/core/construction/ParDos.java | 4 ++-- .../beam/runners/core/construction/SdkComponents.java | 3 ++- .../TransformPayloadTranslatorRegistrar.java | 2 +- .../core/construction/WindowIntoTranslator.java | 2 +- ...sformsTest.java => PTransformTranslationTest.java} | 11 ++++++----- 6 files changed, 14 insertions(+), 12 deletions(-) rename runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/{PTransforms.java => PTransformTranslation.java} (98%) rename runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/{PTransformsTest.java => PTransformTranslationTest.java} (95%) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java rename to runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index 9826b77b300a..86638dec8098 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransforms.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -37,7 +37,7 @@ * Utilities for converting {@link PTransform PTransforms} to and from {@link RunnerApi Runner API * protocol buffers}. */ -public class PTransforms { +public class PTransformTranslation { private static final Map, TransformPayloadTranslator> KNOWN_PAYLOAD_TRANSLATORS = loadTransformPayloadTranslators(); @@ -52,7 +52,7 @@ public class PTransforms { return builder.build(); } - private PTransforms() {} + private PTransformTranslation() {} /** * Translates an {@link AppliedPTransform} into a runner API proto. diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java index 2ecc04161fed..12f29694ca1e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java @@ -34,7 +34,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.construction.PTransforms.TransformPayloadTranslator; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; @@ -96,7 +96,7 @@ public class ParDos { * A {@link TransformPayloadTranslator} for {@link ParDo}. */ public static class ParDoPayloadTranslator - implements PTransforms.TransformPayloadTranslator> { + implements PTransformTranslation.TransformPayloadTranslator> { public static TransformPayloadTranslator create() { return new ParDoPayloadTranslator(); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java index 3d8d4cd3b94e..da229820a984 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java @@ -131,7 +131,8 @@ String registerPTransform( return name; } checkNotNull(children, "child nodes may not be null"); - componentsBuilder.putTransforms(name, PTransforms.toProto(appliedPTransform, children, this)); + componentsBuilder.putTransforms(name, PTransformTranslation + .toProto(appliedPTransform, children, this)); return name; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java index bc568a616159..3b3ffa18b26b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformPayloadTranslatorRegistrar.java @@ -19,7 +19,7 @@ package org.apache.beam.runners.core.construction; import java.util.Map; -import org.apache.beam.runners.core.construction.PTransforms.TransformPayloadTranslator; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.transforms.PTransform; /** A registrar of TransformPayloadTranslator. */ diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java index ea4c9965790c..7ed2a499a95e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java @@ -20,7 +20,7 @@ import com.google.protobuf.Any; import com.google.protobuf.InvalidProtocolBufferException; -import org.apache.beam.runners.core.construction.PTransforms.TransformPayloadTranslator; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java similarity index 95% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformsTest.java rename to runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java index 41255448dee7..0e6ef975c6c8 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java @@ -59,10 +59,10 @@ import org.junit.runners.Parameterized.Parameters; /** - * Tests for {@link PTransforms}. + * Tests for {@link PTransformTranslation}. */ @RunWith(Parameterized.class) -public class PTransformsTest { +public class PTransformTranslationTest { @Parameters(name = "{index}: {0}") public static Iterable data() { @@ -88,7 +88,7 @@ public static Iterable data() { @AutoValue abstract static class ToAndFromProtoSpec { public static ToAndFromProtoSpec leaf(AppliedPTransform transform) { - return new AutoValue_PTransformsTest_ToAndFromProtoSpec( + return new AutoValue_PTransformTranslationTest_ToAndFromProtoSpec( transform, Collections.emptyList()); } @@ -97,7 +97,7 @@ public static ToAndFromProtoSpec composite( List childSpecs = new ArrayList<>(); childSpecs.add(spec); childSpecs.addAll(Arrays.asList(specs)); - return new AutoValue_PTransformsTest_ToAndFromProtoSpec(topLevel, childSpecs); + return new AutoValue_PTransformTranslationTest_ToAndFromProtoSpec(topLevel, childSpecs); } abstract AppliedPTransform getTransform(); @@ -139,7 +139,8 @@ private RunnerApi.PTransform convert(ToAndFromProtoSpec spec, SdkComponents comp // Sanity call components.getExistingPTransformId(child.getTransform()); } - PTransform convert = PTransforms.toProto(spec.getTransform(), childTransforms, components); + PTransform convert = PTransformTranslation + .toProto(spec.getTransform(), childTransforms, components); // Make sure the converted transform is registered. Convert it independently, but if this is a // child spec, the child must be in the components. components.registerPTransform(spec.getTransform(), childTransforms); From 446093836016dabf021d34ca0a858e313f493e2f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 15:28:49 -0700 Subject: [PATCH 057/159] Rename ParDos to ParDoTranslation --- .../{ParDos.java => ParDoTranslation.java} | 4 ++-- ...{ParDosTest.java => ParDoTranslationTest.java} | 15 ++++++++------- 2 files changed, 10 insertions(+), 9 deletions(-) rename runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/{ParDos.java => ParDoTranslation.java} (99%) rename runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/{ParDosTest.java => ParDoTranslationTest.java} (95%) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java rename to runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java index 12f29694ca1e..baed246c7cb8 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDos.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java @@ -73,7 +73,7 @@ /** * Utilities for interacting with {@link ParDo} instances and {@link ParDoPayload} protos. */ -public class ParDos { +public class ParDoTranslation { /** * The URN for a {@link ParDoPayload}. */ @@ -191,7 +191,7 @@ private static TimerSpec toProto(TimerDeclaration timer) { abstract static class DoFnAndMainOutput implements Serializable { public static DoFnAndMainOutput of( DoFn fn, TupleTag tag) { - return new AutoValue_ParDos_DoFnAndMainOutput(fn, tag); + return new AutoValue_ParDoTranslation_DoFnAndMainOutput(fn, tag); } abstract DoFn getDoFn(); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java similarity index 95% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java rename to runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java index b6f0b7d9d68e..ec2795746451 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDosTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java @@ -66,9 +66,9 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; -/** Tests for {@link ParDos}. */ +/** Tests for {@link ParDoTranslation}. */ @RunWith(Parameterized.class) -public class ParDosTest { +public class ParDoTranslationTest { public static TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); private static PCollectionView singletonSideInput = @@ -106,11 +106,12 @@ public class ParDosTest { @Test public void testToAndFromProto() throws Exception { SdkComponents components = SdkComponents.create(); - ParDoPayload payload = ParDos.toProto(parDo, components); + ParDoPayload payload = ParDoTranslation.toProto(parDo, components); - assertThat(ParDos.getDoFn(payload), Matchers.>equalTo(parDo.getFn())); + assertThat(ParDoTranslation.getDoFn(payload), Matchers.>equalTo(parDo.getFn())); assertThat( - ParDos.getMainOutputTag(payload), Matchers.>equalTo(parDo.getMainOutputTag())); + ParDoTranslation.getMainOutputTag(payload), + Matchers.>equalTo(parDo.getMainOutputTag())); for (PCollectionView view : parDo.getSideInputs()) { payload.getSideInputsOrThrow(view.getTagInternal().getId()); } @@ -137,7 +138,7 @@ public void toAndFromTransformProto() throws Exception { for (PCollectionView view : parDo.getSideInputs()) { SideInput sideInput = parDoPayload.getSideInputsOrThrow(view.getTagInternal().getId()); PCollectionView restoredView = - ParDos.fromProto( + ParDoTranslation.fromProto( sideInput, view.getTagInternal().getId(), protoTransform, protoComponents); assertThat(restoredView.getTagInternal(), equalTo(view.getTagInternal())); assertThat(restoredView.getViewFn(), instanceOf(view.getViewFn().getClass())); @@ -151,7 +152,7 @@ public void toAndFromTransformProto() throws Exception { } String mainInputId = components.registerPCollection(mainInput); assertThat( - ParDos.getMainInput(protoTransform, protoComponents), + ParDoTranslation.getMainInput(protoTransform, protoComponents), equalTo(protoComponents.getPcollectionsOrThrow(mainInputId))); } From bc4f44f46ca6afc0018834eb467b1112763c3323 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 15:29:16 -0700 Subject: [PATCH 058/159] Rename WindowIntoTranslator to WindowIntoTranslation --- ...ndowIntoTranslator.java => WindowIntoTranslation.java} | 4 ++-- ...TranslatorTest.java => WindowIntoTranslationTest.java} | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) rename runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/{WindowIntoTranslator.java => WindowIntoTranslation.java} (94%) rename runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/{WindowIntoTranslatorTest.java => WindowIntoTranslationTest.java} (95%) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java similarity index 94% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java rename to runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java index 7ed2a499a95e..69793b56d83c 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslator.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java @@ -33,7 +33,7 @@ * Utility methods for translating a {@link Window.Assign} to and from {@link RunnerApi} * representations. */ -public class WindowIntoTranslator { +public class WindowIntoTranslation { static class WindowAssignTranslator implements TransformPayloadTranslator> { @Override @@ -42,7 +42,7 @@ public FunctionSpec translate( return FunctionSpec.newBuilder() .setUrn("urn:beam:transform:window:v1") .setParameter( - Any.pack(WindowIntoTranslator.toProto(transform.getTransform(), components))) + Any.pack(WindowIntoTranslation.toProto(transform.getTransform(), components))) .build(); } } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java similarity index 95% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.java rename to runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java index eaefe2eade3b..cb9617abd188 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslatorTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java @@ -51,10 +51,10 @@ import org.junit.runners.Parameterized.Parameters; /** - * Tests for {@link WindowIntoTranslator}. + * Tests for {@link WindowIntoTranslation}. */ @RunWith(Parameterized.class) -public class WindowIntoTranslatorTest { +public class WindowIntoTranslationTest { @Parameters(name = "{index}: {0}") public static Iterable> data() { // This pipeline exists for construction, not to run any test. @@ -93,9 +93,9 @@ public void visitPrimitiveTransform(Node node) { SdkComponents components = SdkComponents.create(); WindowIntoPayload payload = - WindowIntoTranslator.toProto(assign.get().getTransform(), components); + WindowIntoTranslation.toProto(assign.get().getTransform(), components); - assertEquals(windowFn, WindowIntoTranslator.getWindowFn(payload)); + assertEquals(windowFn, WindowIntoTranslation.getWindowFn(payload)); } private static class CustomWindows extends PartitioningWindowFn { From b35e91d4ed99b74d37a08a1385018b4ca326b3a0 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 15:30:22 -0700 Subject: [PATCH 059/159] Rename ReadTranslator to ReadTranslation --- .../{ReadTranslator.java => ReadTranslation.java} | 2 +- ...dTranslatorTest.java => ReadTranslationTest.java} | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) rename runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/{ReadTranslator.java => ReadTranslation.java} (99%) rename runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/{ReadTranslatorTest.java => ReadTranslationTest.java} (93%) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslator.java rename to runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java index f94493862b2e..d6c34008f29e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslator.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java @@ -39,7 +39,7 @@ * Methods for translating {@link Read.Bounded} and {@link Read.Unbounded} * {@link PTransform PTransforms} into {@link ReadPayload} protos. */ -public class ReadTranslator { +public class ReadTranslation { private static final String JAVA_SERIALIZED_BOUNDED_SOURCE = "urn:beam:java:boundedsource:v1"; private static final String JAVA_SERIALIZED_UNBOUNDED_SOURCE = "urn:beam:java:unboundedsource:v1"; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslatorTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java similarity index 93% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslatorTest.java rename to runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java index a603e342bc9f..740b3245dd6a 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslatorTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java @@ -51,10 +51,10 @@ import org.junit.runners.Parameterized.Parameters; /** - * Tests for {@link ReadTranslator}. + * Tests for {@link ReadTranslation}. */ @RunWith(Parameterized.class) -public class ReadTranslatorTest { +public class ReadTranslationTest { @Parameters(name = "{index}: {0}") public static Iterable> data() { @@ -74,9 +74,9 @@ public void testToFromProtoBounded() throws Exception { assumeThat(source, instanceOf(BoundedSource.class)); BoundedSource boundedSource = (BoundedSource) this.source; Read.Bounded boundedRead = Read.from(boundedSource); - ReadPayload payload = ReadTranslator.toProto(boundedRead); + ReadPayload payload = ReadTranslation.toProto(boundedRead); assertThat(payload.getIsBounded(), equalTo(RunnerApi.IsBounded.BOUNDED)); - BoundedSource deserializedSource = ReadTranslator.boundedSourceFromProto(payload); + BoundedSource deserializedSource = ReadTranslation.boundedSourceFromProto(payload); assertThat(deserializedSource, Matchers.>equalTo(source)); } @@ -85,9 +85,9 @@ public void testToFromProtoUnbounded() throws Exception { assumeThat(source, instanceOf(UnboundedSource.class)); UnboundedSource unboundedSource = (UnboundedSource) this.source; Read.Unbounded unboundedRead = Read.from(unboundedSource); - ReadPayload payload = ReadTranslator.toProto(unboundedRead); + ReadPayload payload = ReadTranslation.toProto(unboundedRead); assertThat(payload.getIsBounded(), equalTo(RunnerApi.IsBounded.UNBOUNDED)); - UnboundedSource deserializedSource = ReadTranslator.unboundedSourceFromProto(payload); + UnboundedSource deserializedSource = ReadTranslation.unboundedSourceFromProto(payload); assertThat(deserializedSource, Matchers.>equalTo(source)); } From 7e37b70317bd06f300a5423cc4cb76a06c3955c3 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 15:31:49 -0700 Subject: [PATCH 060/159] Rename Coders to CoderTranslation --- .../{Coders.java => CoderTranslation.java} | 2 +- .../construction/PCollectionTranslation.java | 3 ++- .../core/construction/ParDoTranslation.java | 3 ++- .../core/construction/SdkComponents.java | 2 +- ...ersTest.java => CoderTranslationTest.java} | 19 ++++++++++--------- 5 files changed, 16 insertions(+), 13 deletions(-) rename runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/{Coders.java => CoderTranslation.java} (99%) rename runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/{CodersTest.java => CoderTranslationTest.java} (89%) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Coders.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Coders.java rename to runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java index 6c2caa9ba0c0..470db6ab8c75 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Coders.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java @@ -49,7 +49,7 @@ import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; /** Converts to and from Beam Runner API representations of {@link Coder Coders}. */ -public class Coders { +public class CoderTranslation { // This URN says that the coder is just a UDF blob this SDK understands // TODO: standardize such things public static final String JAVA_SERIALIZED_CODER_URN = "urn:beam:coders:javasdk:0.1"; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java index cad7b97c74b5..46f714e66ef5 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java @@ -53,7 +53,8 @@ public static IsBounded isBounded(RunnerApi.PCollection pCollection) { public static Coder getCoder( RunnerApi.PCollection pCollection, RunnerApi.Components components) throws IOException { - return Coders.fromProto(components.getCodersOrThrow(pCollection.getCoderId()), components); + return CoderTranslation + .fromProto(components.getCodersOrThrow(pCollection.getCoderId()), components); } public static WindowingStrategy getWindowingStrategy( diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java index baed246c7cb8..bc5bb0e414a3 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java @@ -269,7 +269,8 @@ public static PCollectionView fromProto( components.getWindowingStrategiesOrThrow(inputCollection.getWindowingStrategyId()), components); Coder elemCoder = - Coders.fromProto(components.getCodersOrThrow(inputCollection.getCoderId()), components); + CoderTranslation + .fromProto(components.getCodersOrThrow(inputCollection.getCoderId()), components); Coder>> coder = (Coder) IterableCoder.of( diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java index da229820a984..5c8187513402 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java @@ -222,7 +222,7 @@ String registerCoder(Coder coder) throws IOException { String baseName = NameUtils.approximateSimpleName(coder); String name = uniqify(baseName, coderIds.values()); coderIds.put(Equivalence.identity().wrap(coder), name); - RunnerApi.Coder coderProto = Coders.toProto(coder, this); + RunnerApi.Coder coderProto = CoderTranslation.toProto(coder, this); componentsBuilder.putCoders(name, coderProto); return name; } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CodersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java similarity index 89% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CodersTest.java rename to runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java index 42fba7cda508..39549d062ed6 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CodersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java @@ -56,9 +56,9 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; -/** Tests for {@link Coders}. */ +/** Tests for {@link CoderTranslation}. */ @RunWith(Enclosed.class) -public class CodersTest { +public class CoderTranslationTest { private static final Set> KNOWN_CODERS = ImmutableSet.>builder() .add(ByteArrayCoder.of()) @@ -84,7 +84,8 @@ public void validateKnownCoders() { // Validates that every known coder in the Coders class is represented in a "Known Coder" // tests, which demonstrates that they are serialized via components and specified URNs rather // than java serialized - Set> knownCoderClasses = Coders.KNOWN_CODER_URNS.keySet(); + Set> knownCoderClasses = + CoderTranslation.KNOWN_CODER_URNS.keySet(); Set> knownCoderTests = new HashSet<>(); for (StructuredCoder coder : KNOWN_CODERS) { knownCoderTests.add(coder.getClass()); @@ -94,7 +95,7 @@ public void validateKnownCoders() { assertThat( String.format( "Missing validation of known coder %s in %s", - missingKnownCoders, CodersTest.class.getSimpleName()), + missingKnownCoders, CoderTranslationTest.class.getSimpleName()), missingKnownCoders, Matchers.empty()); } @@ -103,8 +104,8 @@ public void validateKnownCoders() { public void validateCoderTranslators() { assertThat( "Every Known Coder must have a Known Translator", - Coders.KNOWN_CODER_URNS.keySet(), - equalTo(Coders.KNOWN_TRANSLATORS.keySet())); + CoderTranslation.KNOWN_CODER_URNS.keySet(), + equalTo(CoderTranslation.KNOWN_TRANSLATORS.keySet())); } } @@ -132,17 +133,17 @@ public static Iterable> data() { @Test public void toAndFromProto() throws Exception { SdkComponents componentsBuilder = SdkComponents.create(); - RunnerApi.Coder coderProto = Coders.toProto(coder, componentsBuilder); + RunnerApi.Coder coderProto = CoderTranslation.toProto(coder, componentsBuilder); Components encodedComponents = componentsBuilder.toComponents(); - Coder decodedCoder = Coders.fromProto(coderProto, encodedComponents); + Coder decodedCoder = CoderTranslation.fromProto(coderProto, encodedComponents); assertThat(decodedCoder, Matchers.>equalTo(coder)); if (KNOWN_CODERS.contains(coder)) { for (RunnerApi.Coder encodedCoder : encodedComponents.getCodersMap().values()) { assertThat( encodedCoder.getSpec().getSpec().getUrn(), - not(equalTo(Coders.JAVA_SERIALIZED_CODER_URN))); + not(equalTo(CoderTranslation.JAVA_SERIALIZED_CODER_URN))); } } } From c8b2119ab9a75c7f781ce73ea9352734640a6f46 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 15:32:47 -0700 Subject: [PATCH 061/159] Rename WindowingStrategies to WindowingStrategyTranslation --- .../core/construction/PCollectionTranslation.java | 2 +- .../runners/core/construction/ParDoTranslation.java | 2 +- .../beam/runners/core/construction/SdkComponents.java | 2 +- .../core/construction/WindowIntoTranslation.java | 4 ++-- ...ategies.java => WindowingStrategyTranslation.java} | 2 +- ...est.java => WindowingStrategyTranslationTest.java} | 11 ++++++----- .../runners/dataflow/DataflowPipelineTranslator.java | 4 ++-- 7 files changed, 14 insertions(+), 13 deletions(-) rename runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/{WindowingStrategies.java => WindowingStrategyTranslation.java} (99%) rename runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/{WindowingStrategiesTest.java => WindowingStrategyTranslationTest.java} (91%) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java index 46f714e66ef5..303c02d44a11 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java @@ -60,7 +60,7 @@ public static Coder getCoder( public static WindowingStrategy getWindowingStrategy( RunnerApi.PCollection pCollection, RunnerApi.Components components) throws InvalidProtocolBufferException { - return WindowingStrategies.fromProto( + return WindowingStrategyTranslation.fromProto( components.getWindowingStrategiesOrThrow(pCollection.getWindowingStrategyId()), components); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java index bc5bb0e414a3..28d577ff2211 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java @@ -265,7 +265,7 @@ public static PCollectionView fromProto( RunnerApi.PCollection inputCollection = components.getPcollectionsOrThrow(parDoTransform.getInputsOrThrow(id)); WindowingStrategy windowingStrategy = - WindowingStrategies.fromProto( + WindowingStrategyTranslation.fromProto( components.getWindowingStrategiesOrThrow(inputCollection.getWindowingStrategyId()), components); Coder elemCoder = diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java index 5c8187513402..b0f164f4bcbd 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java @@ -200,7 +200,7 @@ String registerWindowingStrategy(WindowingStrategy windowingStrategy) thro String name = uniqify(baseName, windowingStrategyIds.values()); windowingStrategyIds.put(windowingStrategy, name); RunnerApi.WindowingStrategy windowingStrategyProto = - WindowingStrategies.toProto(windowingStrategy, this); + WindowingStrategyTranslation.toProto(windowingStrategy, this); componentsBuilder.putWindowingStrategies(name, windowingStrategyProto); return name; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java index 69793b56d83c..215beba43cd8 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java @@ -49,13 +49,13 @@ public FunctionSpec translate( public static WindowIntoPayload toProto(Window.Assign transform, SdkComponents components) { return WindowIntoPayload.newBuilder() - .setWindowFn(WindowingStrategies.toProto(transform.getWindowFn(), components)) + .setWindowFn(WindowingStrategyTranslation.toProto(transform.getWindowFn(), components)) .build(); } public static WindowFn getWindowFn(WindowIntoPayload payload) throws InvalidProtocolBufferException { SdkFunctionSpec spec = payload.getWindowFn(); - return WindowingStrategies.windowFnFromProto(spec); + return WindowingStrategyTranslation.windowFnFromProto(spec); } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategies.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java similarity index 99% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategies.java rename to runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java index 8dceebbe6cc3..061f30961fd8 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategies.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java @@ -40,7 +40,7 @@ import org.joda.time.Duration; /** Utilities for working with {@link WindowingStrategy WindowingStrategies}. */ -public class WindowingStrategies implements Serializable { +public class WindowingStrategyTranslation implements Serializable { public static AccumulationMode fromProto(RunnerApi.AccumulationMode proto) { switch (proto) { diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategiesTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java similarity index 91% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategiesTest.java rename to runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java index 7296a7789943..1e528039562c 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategiesTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java @@ -41,7 +41,7 @@ /** Unit tests for {@link WindowingStrategy}. */ @RunWith(Parameterized.class) -public class WindowingStrategiesTest { +public class WindowingStrategyTranslationTest { // Each spec activates tests of all subsets of its fields @AutoValue @@ -50,7 +50,7 @@ abstract static class ToProtoAndBackSpec { } private static ToProtoAndBackSpec toProtoAndBackSpec(WindowingStrategy windowingStrategy) { - return new AutoValue_WindowingStrategiesTest_ToProtoAndBackSpec(windowingStrategy); + return new AutoValue_WindowingStrategyTranslationTest_ToProtoAndBackSpec(windowingStrategy); } private static final WindowFn REPRESENTATIVE_WINDOW_FN = @@ -85,7 +85,8 @@ public static Iterable data() { public void testToProtoAndBack() throws Exception { WindowingStrategy windowingStrategy = toProtoAndBackSpec.getWindowingStrategy(); WindowingStrategy toProtoAndBackWindowingStrategy = - WindowingStrategies.fromProto(WindowingStrategies.toProto(windowingStrategy)); + WindowingStrategyTranslation.fromProto( + WindowingStrategyTranslation.toProto(windowingStrategy)); assertThat( toProtoAndBackWindowingStrategy, @@ -97,11 +98,11 @@ public void testToProtoAndBackWithComponents() throws Exception { WindowingStrategy windowingStrategy = toProtoAndBackSpec.getWindowingStrategy(); SdkComponents components = SdkComponents.create(); RunnerApi.WindowingStrategy proto = - WindowingStrategies.toProto(windowingStrategy, components); + WindowingStrategyTranslation.toProto(windowingStrategy, components); RunnerApi.Components protoComponents = components.toComponents(); assertThat( - WindowingStrategies.fromProto(proto, protoComponents).fixDefaults(), + WindowingStrategyTranslation.fromProto(proto, protoComponents).fixDefaults(), Matchers.>equalTo(windowingStrategy.fixDefaults())); protoComponents.getCodersOrThrow( 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 index 6d7a0f847ca2..af93ef59bba4 100644 --- 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 @@ -56,7 +56,7 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import javax.annotation.Nullable; -import org.apache.beam.runners.core.construction.WindowingStrategies; +import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.runners.dataflow.BatchViewOverrides.GroupByKeyAndSortValuesOnly; import org.apache.beam.runners.dataflow.DataflowRunner.CombineGroupedValues; import org.apache.beam.runners.dataflow.PrimitiveParDoSingleFactory.ParDoSingle; @@ -124,7 +124,7 @@ public class DataflowPipelineTranslator { private static byte[] serializeWindowingStrategy(WindowingStrategy windowingStrategy) { try { - return WindowingStrategies.toProto(windowingStrategy).toByteArray(); + return WindowingStrategyTranslation.toProto(windowingStrategy).toByteArray(); } catch (Exception e) { throw new RuntimeException( String.format("Unable to format windowing strategy %s as bytes", windowingStrategy), e); From 4fa38e2d590cc4472c119b57790f7a724a700e43 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 15:33:33 -0700 Subject: [PATCH 062/159] Rename Triggers to TriggerTranslation --- .../translation/operators/ApexGroupByKeyOperator.java | 4 ++-- .../{Triggers.java => TriggerTranslation.java} | 6 +++--- .../construction/WindowingStrategyTranslation.java | 4 ++-- .../{TriggersTest.java => TriggerTranslationTest.java} | 9 +++++---- .../core/GroupAlsoByWindowViaWindowSetNewDoFn.java | 4 ++-- .../org/apache/beam/runners/core/ReduceFnTester.java | 10 ++++++---- .../direct/GroupAlsoByWindowEvaluatorFactory.java | 4 ++-- .../stateful/SparkGroupAlsoByWindowViaWindowSet.java | 4 ++-- .../SparkGroupAlsoByWindowViaOutputBufferFn.java | 4 ++-- 9 files changed, 26 insertions(+), 23 deletions(-) rename runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/{Triggers.java => TriggerTranslation.java} (98%) rename runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/{TriggersTest.java => TriggerTranslationTest.java} (94%) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java index 1d48e2032d8a..39f681fcb821 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexGroupByKeyOperator.java @@ -41,7 +41,7 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.runners.core.construction.Triggers; +import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.coders.Coder; @@ -163,7 +163,7 @@ private ReduceFnRunner, BoundedWindow> newReduceFnRunner(K key windowingStrategy, ExecutableTriggerStateMachine.create( TriggerStateMachines.stateMachineForTrigger( - Triggers.toProto(windowingStrategy.getTrigger()))), + TriggerTranslation.toProto(windowingStrategy.getTrigger()))), stateInternalsFactory.stateInternalsForKey(key), timerInternals, new OutputWindowedValue>>() { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Triggers.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TriggerTranslation.java similarity index 98% rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Triggers.java rename to runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TriggerTranslation.java index df6c9ed1fa04..777b165a9b51 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Triggers.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TriggerTranslation.java @@ -47,9 +47,9 @@ import org.joda.time.Duration; import org.joda.time.Instant; -/** Utilities for working with {@link Triggers Triggers}. */ +/** Utilities for working with {@link TriggerTranslation Triggers}. */ @Experimental(Experimental.Kind.TRIGGER) -public class Triggers implements Serializable { +public class TriggerTranslation implements Serializable { @VisibleForTesting static final ProtoConverter CONVERTER = new ProtoConverter(); @@ -332,5 +332,5 @@ private static List protosToTriggers(List triggers) } // Do not instantiate - private Triggers() {} + private TriggerTranslation() {} } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java index 061f30961fd8..e92565f1a783 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java @@ -210,7 +210,7 @@ public static RunnerApi.WindowingStrategy toProto( .setAccumulationMode(toProto(windowingStrategy.getMode())) .setClosingBehavior(toProto(windowingStrategy.getClosingBehavior())) .setAllowedLateness(windowingStrategy.getAllowedLateness().getMillis()) - .setTrigger(Triggers.toProto(windowingStrategy.getTrigger())) + .setTrigger(TriggerTranslation.toProto(windowingStrategy.getTrigger())) .setWindowFn(windowFnSpec) .setWindowCoderId( components.registerCoder(windowingStrategy.getWindowFn().windowCoder())); @@ -247,7 +247,7 @@ public static RunnerApi.WindowingStrategy toProto( WindowFn windowFn = windowFnFromProto(windowFnSpec); TimestampCombiner timestampCombiner = timestampCombinerFromProto(proto.getOutputTime()); AccumulationMode accumulationMode = fromProto(proto.getAccumulationMode()); - Trigger trigger = Triggers.fromProto(proto.getTrigger()); + Trigger trigger = TriggerTranslation.fromProto(proto.getTrigger()); ClosingBehavior closingBehavior = fromProto(proto.getClosingBehavior()); Duration allowedLateness = Duration.millis(proto.getAllowedLateness()); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggerTranslationTest.java similarity index 94% rename from runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggersTest.java rename to runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggerTranslationTest.java index cf9d40c8c9cf..55ea87bca9e3 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggerTranslationTest.java @@ -41,9 +41,9 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; -/** Tests for utilities in {@link Triggers}. */ +/** Tests for utilities in {@link TriggerTranslation}. */ @RunWith(Parameterized.class) -public class TriggersTest { +public class TriggerTranslationTest { @AutoValue abstract static class ToProtoAndBackSpec { @@ -51,7 +51,7 @@ abstract static class ToProtoAndBackSpec { } private static ToProtoAndBackSpec toProtoAndBackSpec(Trigger trigger) { - return new AutoValue_TriggersTest_ToProtoAndBackSpec(trigger); + return new AutoValue_TriggerTranslationTest_ToProtoAndBackSpec(trigger); } @Parameters(name = "{index}: {0}") @@ -104,7 +104,8 @@ public static Iterable data() { @Test public void testToProtoAndBack() throws Exception { Trigger trigger = toProtoAndBackSpec.getTrigger(); - Trigger toProtoAndBackTrigger = Triggers.fromProto(Triggers.toProto(trigger)); + Trigger toProtoAndBackTrigger = + TriggerTranslation.fromProto(TriggerTranslation.toProto(trigger)); assertThat(toProtoAndBackTrigger, equalTo(trigger)); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java index 744d1625667d..0a520bdb513d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.core; import java.util.Collection; -import org.apache.beam.runners.core.construction.Triggers; +import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.transforms.DoFn; @@ -122,7 +122,7 @@ public void processElement(ProcessContext c) throws Exception { windowingStrategy, ExecutableTriggerStateMachine.create( TriggerStateMachines.stateMachineForTrigger( - Triggers.toProto(windowingStrategy.getTrigger()))), + TriggerTranslation.toProto(windowingStrategy.getTrigger()))), stateInternals, timerInternals, outputWindowedValue(), diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index 7de8f3b28dfd..7f83eae78724 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -38,7 +38,7 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.runners.core.construction.Triggers; +import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachineRunner; @@ -116,7 +116,7 @@ public class ReduceFnTester { return new ReduceFnTester, W>( windowingStrategy, TriggerStateMachines.stateMachineForTrigger( - Triggers.toProto(windowingStrategy.getTrigger())), + TriggerTranslation.toProto(windowingStrategy.getTrigger())), SystemReduceFn.buffering(VarIntCoder.of()), IterableCoder.of(VarIntCoder.of()), PipelineOptionsFactory.create(), @@ -179,7 +179,8 @@ ReduceFnTester combining( return combining( strategy, - TriggerStateMachines.stateMachineForTrigger(Triggers.toProto(strategy.getTrigger())), + TriggerStateMachines.stateMachineForTrigger( + TriggerTranslation.toProto(strategy.getTrigger())), combineFn, outputCoder); } @@ -227,7 +228,8 @@ ReduceFnTester combining( return combining( strategy, - TriggerStateMachines.stateMachineForTrigger(Triggers.toProto(strategy.getTrigger())), + TriggerStateMachines.stateMachineForTrigger( + TriggerTranslation.toProto(strategy.getTrigger())), combineFn, outputCoder, options, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index 1a588eedf895..a944e758cc6d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -32,7 +32,7 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.UnsupportedSideInputReader; -import org.apache.beam.runners.core.construction.Triggers; +import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; @@ -162,7 +162,7 @@ public void processElement(WindowedValue> element) throws Ex (CopyOnAccessInMemoryStateInternals) stepContext.stateInternals(); DirectTimerInternals timerInternals = stepContext.timerInternals(); RunnerApi.Trigger runnerApiTrigger = - Triggers.toProto(windowingStrategy.getTrigger()); + TriggerTranslation.toProto(windowingStrategy.getTrigger()); ReduceFnRunner, BoundedWindow> reduceFnRunner = new ReduceFnRunner<>( key, diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java index 815b6ba5dc02..be4f3f65a3b7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java @@ -31,7 +31,7 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.UnsupportedSideInputReader; -import org.apache.beam.runners.core.construction.Triggers; +import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.metrics.CounterCell; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; @@ -260,7 +260,7 @@ public JavaPairRDD call( windowingStrategy, ExecutableTriggerStateMachine.create( TriggerStateMachines.stateMachineForTrigger( - Triggers.toProto(windowingStrategy.getTrigger()))), + TriggerTranslation.toProto(windowingStrategy.getTrigger()))), stateInternals, timerInternals, outputHolder, diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java index be023356ba54..d2a34244e6e5 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java @@ -30,7 +30,7 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.UnsupportedSideInputReader; -import org.apache.beam.runners.core.construction.Triggers; +import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.runners.spark.aggregators.NamedAggregators; @@ -92,7 +92,7 @@ public Iterable>>> call( windowingStrategy, ExecutableTriggerStateMachine.create( TriggerStateMachines.stateMachineForTrigger( - Triggers.toProto(windowingStrategy.getTrigger()))), + TriggerTranslation.toProto(windowingStrategy.getTrigger()))), stateInternals, timerInternals, outputter, From efc8f425317c9c54912c78af8fbbe55586aafc1b Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Mon, 22 May 2017 16:40:09 -0700 Subject: [PATCH 063/159] upgrading python sdk dependencies --- sdks/python/setup.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 9bf3cf4d37e9..befc0249c0bc 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -94,7 +94,7 @@ def get_version(): 'httplib2>=0.8,<0.10', 'mock>=1.0.1,<3.0.0', 'oauth2client>=2.0.1,<4.0.0', - 'protobuf==3.2.0', + 'protobuf>=3.2.0,<=3.3.0', 'pyyaml>=3.12,<4.0.0', ] @@ -103,11 +103,11 @@ def get_version(): ] GCP_REQUIREMENTS = [ - 'google-apitools==0.5.10', - 'proto-google-cloud-datastore-v1==0.90.0', + 'google-apitools>=0.5.10,<=0.5.11', + 'proto-google-cloud-datastore-v1>=0.90.0,<=0.90.4', 'googledatastore==7.0.1', # GCP packages required by tests - 'google-cloud-bigquery>=0.23.0,<0.24.0', + 'google-cloud-bigquery>=0.23.0,<0.25.0', ] From 7fb2938e66e48160e9df878ccf9d0c2a67790151 Mon Sep 17 00:00:00 2001 From: Borisa Zivkovic Date: Tue, 16 May 2017 14:16:07 +0100 Subject: [PATCH 064/159] Do not ever shrink allowed lateness This does not seem to have any use except to cause data loss. --- .../beam/sdk/transforms/windowing/Window.java | 6 +++- .../sdk/transforms/windowing/WindowTest.java | 36 +++++++++++++++++++ 2 files changed, 41 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java index dc4863b06b0c..105ebfbe24af 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java @@ -19,6 +19,7 @@ import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Ordering; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; @@ -141,6 +142,7 @@ */ @AutoValue public abstract class Window extends PTransform, PCollection> { + /** * Specifies the conditions under which a final pane will be created when a window is permanently * closed. @@ -313,7 +315,8 @@ public Window withAllowedLateness(Duration allowedLateness, ClosingBehavior b result = result.withMode(getAccumulationMode()); } if (getAllowedLateness() != null) { - result = result.withAllowedLateness(getAllowedLateness()); + result = result.withAllowedLateness(Ordering.natural().max(getAllowedLateness(), + inputStrategy.getAllowedLateness())); } if (getClosingBehavior() != null) { result = result.withClosingBehavior(getClosingBehavior()); @@ -366,6 +369,7 @@ public PCollection expand(PCollection input) { WindowingStrategy outputStrategy = getOutputStrategyInternal(input.getWindowingStrategy()); + if (getWindowFn() == null) { // A new PCollection must be created in case input is reused in a different location as the // two PCollections will, in general, have a different windowing strategy. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java index f536a9a9b8a1..65af7a1b377c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java @@ -165,6 +165,42 @@ public void testWindowIntoPropagatesLateness() { assertEquals(fixed25, strategy.getWindowFn()); } + @Test + public void testWindowIntoAssignesLongerAllowedLateness() { + + FixedWindows fixed10 = FixedWindows.of(Duration.standardMinutes(10)); + FixedWindows fixed25 = FixedWindows.of(Duration.standardMinutes(25)); + + PCollection notChanged = pipeline + .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of())) + .apply("WindowInto25", Window.into(fixed25) + .withAllowedLateness(Duration.standardDays(1)) + .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(5))) + .accumulatingFiredPanes()) + .apply("WindowInto10", Window.into(fixed10) + .withAllowedLateness(Duration.standardDays(2))); + + assertEquals(Duration.standardDays(2), notChanged.getWindowingStrategy() + .getAllowedLateness()); + + PCollection data = pipeline + .apply("createChanged", Create.of("hello", "world").withCoder(StringUtf8Coder.of())); + + PCollection longWindow = data.apply("WindowInto25c", Window.into(fixed25) + .withAllowedLateness(Duration.standardDays(1)) + .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(5))) + .accumulatingFiredPanes()); + + assertEquals(Duration.standardDays(1), longWindow.getWindowingStrategy() + .getAllowedLateness()); + + PCollection autoCorrectedWindow = longWindow.apply("WindowInto10c", + Window.into(fixed10).withAllowedLateness(Duration.standardHours(1))); + + assertEquals(Duration.standardDays(1), autoCorrectedWindow.getWindowingStrategy() + .getAllowedLateness()); + } + /** * With {@link #testWindowIntoNullWindowFnNoAssign()}, demonstrates that the expansions of the * {@link Window} transform depends on if it actually assigns elements to windows. From 6a792f3251904f191ff27e58bb9bb78b9c30fdd9 Mon Sep 17 00:00:00 2001 From: Daniel Mills Date: Mon, 8 May 2017 16:45:44 -0700 Subject: [PATCH 065/159] Adds large key tests to GroupByKeyTest --- runners/direct-java/pom.xml | 4 +- runners/flink/pom.xml | 2 + runners/google-cloud-dataflow-java/pom.xml | 1 + .../apache/beam/sdk/testing/LargeKeys.java | 48 ++++++++++++ .../beam/sdk/transforms/GroupByKeyTest.java | 74 +++++++++++++++++++ 5 files changed, 128 insertions(+), 1 deletion(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/LargeKeys.java diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 857dcf42fd7b..bec21139d989 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -144,11 +144,13 @@ org.apache.beam.sdk.testing.NeedsRunner + + org.apache.beam.sdk.testing.LargeKeys$Above100MB none true org.apache.beam:beam-sdks-java-core - org.apache.beam:beam-runners-core-java + org.apache.beam:beam-runners-java-core diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index ff73ec1eb9d0..fb0a67c58ebc 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -57,6 +57,7 @@ org.apache.beam.sdk.testing.ValidatesRunner org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders, + org.apache.beam.sdk.testing.LargeKeys$Above100MB, org.apache.beam.sdk.testing.UsesSplittableParDo, org.apache.beam.sdk.testing.UsesCommittedMetrics, org.apache.beam.sdk.testing.UsesTestStream @@ -89,6 +90,7 @@ org.apache.beam.sdk.testing.ValidatesRunner org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders, + org.apache.beam.sdk.testing.LargeKeys$Above100MB, org.apache.beam.sdk.testing.UsesSetState, org.apache.beam.sdk.testing.UsesMapState, org.apache.beam.sdk.testing.UsesCommittedMetrics, diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 895a8e6a513b..16e12664918c 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -217,6 +217,7 @@ validates-runner-tests + org.apache.beam.sdk.testing.LargeKeys$Above10MB, org.apache.beam.sdk.testing.UsesDistributionMetrics, org.apache.beam.sdk.testing.UsesGaugeMetrics, org.apache.beam.sdk.testing.UsesSetState, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/LargeKeys.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/LargeKeys.java new file mode 100644 index 000000000000..384b2980f47c --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/LargeKeys.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing; + +/** + * Category tags for tests which validate that a Beam runner can handle keys up to a given size. + */ +public interface LargeKeys { + /** + * Tests if a runner supports 10KB keys. + */ + public interface Above10KB {} + + /** + * Tests if a runner supports 100KB keys. + */ + public interface Above100KB extends Above10KB {} + + /** + * Tests if a runner supports 1MB keys. + */ + public interface Above1MB extends Above100KB {} + + /** + * Tests if a runner supports 10MB keys. + */ + public interface Above10MB extends Above1MB {} + + /** + * Tests if a runner supports 100MB keys. + */ + public interface Above100MB extends Above10MB {} +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java index 0cd885ca2557..171171f33cd4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java @@ -45,6 +45,7 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.MapCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.testing.LargeKeys; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -427,6 +428,79 @@ public void testGroupByKeyWithBadEqualsHashCode() throws Exception { p.run(); } + private static String bigString(char c, int size) { + char[] buf = new char[size]; + for (int i = 0; i < size; i++) { + buf[i] = c; + } + return new String(buf); + } + + private static void runLargeKeysTest(TestPipeline p, final int keySize) throws Exception { + PCollection> result = p + .apply(Create.of("a", "a", "b")) + .apply("Expand", ParDo.of(new DoFn>() { + @ProcessElement + public void process(ProcessContext c) { + c.output(KV.of(bigString(c.element().charAt(0), keySize), c.element())); + } + })) + .apply(GroupByKey.create()) + .apply("Count", ParDo.of(new DoFn>, KV>() { + @ProcessElement + public void process(ProcessContext c) { + int size = 0; + for (String value : c.element().getValue()) { + size++; + } + c.output(KV.of(c.element().getKey(), size)); + } + })); + + PAssert.that(result).satisfies( + new SerializableFunction>, Void>() { + @Override + public Void apply(Iterable> values) { + assertThat(values, + containsInAnyOrder( + KV.of(bigString('a', keySize), 2), KV.of(bigString('b', keySize), 1))); + return null; + } + }); + + p.run(); + } + + @Test + @Category({ValidatesRunner.class, LargeKeys.Above10KB.class}) + public void testLargeKeys10KB() throws Exception { + runLargeKeysTest(p, 10 << 10); + } + + @Test + @Category({ValidatesRunner.class, LargeKeys.Above100KB.class}) + public void testLargeKeys100KB() throws Exception { + runLargeKeysTest(p, 100 << 10); + } + + @Test + @Category({ValidatesRunner.class, LargeKeys.Above1MB.class}) + public void testLargeKeys1MB() throws Exception { + runLargeKeysTest(p, 1 << 20); + } + + @Test + @Category({ValidatesRunner.class, LargeKeys.Above10MB.class}) + public void testLargeKeys10MB() throws Exception { + runLargeKeysTest(p, 10 << 20); + } + + @Test + @Category({ValidatesRunner.class, LargeKeys.Above100MB.class}) + public void testLargeKeys100MB() throws Exception { + runLargeKeysTest(p, 100 << 20); + } + /** * This is a bogus key class that returns random hash values from {@link #hashCode()} and always * returns {@code false} for {@link #equals(Object)}. The results of the test are correct if From 5b899a8518cc8910f0a855303c14088d72b332e5 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 18 May 2017 10:23:16 -0700 Subject: [PATCH 066/159] Add CombineTranslation This translates Combines to CombinePayloads and back --- .../core/construction/CombineTranslation.java | 125 +++++++++++++++++ .../construction/CombineTranslationTest.java | 130 ++++++++++++++++++ .../org/apache/beam/sdk/transforms/Count.java | 10 ++ .../org/apache/beam/sdk/transforms/Sum.java | 30 ++++ 4 files changed, 295 insertions(+) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java new file mode 100644 index 000000000000..e0b6d5c9abf8 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.common.collect.Iterables; +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.CombinePayload; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SideInput; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.AppliedCombineFn; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +/** + * Methods for translating between {@link Combine.PerKey} {@link PTransform PTransforms} and {@link + * RunnerApi.CombinePayload} protos. + */ +public class CombineTranslation { + private static final String JAVA_SERIALIZED_COMBINE_FN_URN = "urn:beam:java:combinefn:v1"; + + public static CombinePayload toProto( + AppliedPTransform> combine, SdkComponents sdkComponents) + throws IOException { + GlobalCombineFn combineFn = combine.getTransform().getFn(); + try { + Coder accumulatorCoder = extractAccumulatorCoder(combineFn, (AppliedPTransform) combine); + Map sideInputs = new HashMap<>(); + return RunnerApi.CombinePayload.newBuilder() + .setAccumulatorCoderId(sdkComponents.registerCoder(accumulatorCoder)) + .putAllSideInputs(sideInputs) + .setCombineFn(toProto(combineFn)) + .build(); + } catch (CannotProvideCoderException e) { + throw new IllegalStateException(e); + } + } + + private static Coder extractAccumulatorCoder( + GlobalCombineFn combineFn, + AppliedPTransform>, ?, Combine.PerKey> transform) + throws CannotProvideCoderException { + KvCoder inputCoder = + (KvCoder) + ((PCollection>) Iterables.getOnlyElement(transform.getInputs().values())) + .getCoder(); + return AppliedCombineFn.withInputCoder( + combineFn, + transform.getPipeline().getCoderRegistry(), + inputCoder, + transform.getTransform().getSideInputs(), + ((PCollection) Iterables.getOnlyElement(transform.getOutputs().values())) + .getWindowingStrategy()) + .getAccumulatorCoder(); + } + + private static SdkFunctionSpec toProto(GlobalCombineFn combineFn) { + return SdkFunctionSpec.newBuilder() + // TODO: Set Java SDK Environment URN + .setSpec( + FunctionSpec.newBuilder() + .setUrn(JAVA_SERIALIZED_COMBINE_FN_URN) + .setParameter( + Any.pack( + BytesValue.newBuilder() + .setValue( + ByteString.copyFrom( + SerializableUtils.serializeToByteArray(combineFn))) + .build()))) + .build(); + } + + public static Coder getAccumulatorCoder( + CombinePayload payload, RunnerApi.Components components) throws IOException { + String id = payload.getAccumulatorCoderId(); + return Coders.fromProto(components.getCodersOrThrow(id), components); + } + + public static GlobalCombineFn getCombineFn(CombinePayload payload) + throws IOException { + checkArgument(payload.getCombineFn().getSpec().getUrn().equals(JAVA_SERIALIZED_COMBINE_FN_URN)); + return (GlobalCombineFn) + SerializableUtils.deserializeFromByteArray( + payload + .getCombineFn() + .getSpec() + .getParameter() + .unpack(BytesValue.class) + .getValue() + .toByteArray(), + "CombineFn"); + } +} diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java new file mode 100644 index 000000000000..6251545b9b7c --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import static com.google.common.base.Preconditions.checkState; +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.ImmutableList; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.beam.sdk.Pipeline.PipelineVisitor; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.CombinePayload; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.runners.TransformHierarchy.Node; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Combine.BinaryCombineIntegerFn; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; + +/** + * Tests for {@link CombineTranslation}. + */ +@RunWith(Parameterized.class) +public class CombineTranslationTest { + @Parameters(name = "{index}: {0}") + public static Iterable> params() { + BinaryCombineIntegerFn sum = Sum.ofIntegers(); + CombineFn count = Count.combineFn(); + TestCombineFn test = new TestCombineFn(); + return ImmutableList.>builder().add(sum).add(count).add(test).build(); + } + + @Rule public TestPipeline pipeline = TestPipeline.create(); + @Parameter(0) + public Combine.CombineFn combineFn; + + @Test + public void testToFromProto() throws Exception { + PCollection input = pipeline.apply(Create.of(1, 2, 3)); + input.apply(Combine.globally(combineFn)); + final AtomicReference>> combine = + new AtomicReference<>(); + pipeline.traverseTopologically( + new PipelineVisitor.Defaults() { + @Override + public void leaveCompositeTransform(Node node) { + if (node.getTransform() instanceof Combine.PerKey) { + checkState(combine.get() == null); + combine.set((AppliedPTransform) node.toAppliedPTransform(getPipeline())); + } + } + }); + checkState(combine.get() != null); + + SdkComponents sdkComponents = SdkComponents.create(); + CombinePayload combineProto = CombineTranslation.toProto(combine.get(), sdkComponents); + RunnerApi.Components componentsProto = sdkComponents.toComponents(); + + assertEquals( + combineFn.getAccumulatorCoder(pipeline.getCoderRegistry(), input.getCoder()), + CombineTranslation.getAccumulatorCoder(combineProto, componentsProto)); + assertEquals(combineFn, CombineTranslation.getCombineFn(combineProto)); + } + + private static class TestCombineFn extends Combine.CombineFn { + @Override + public Void createAccumulator() { + return null; + } + + @Override + public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { + return (Coder) VoidCoder.of(); + } + + @Override + public Void extractOutput(Void accumulator) { + return accumulator; + } + + @Override + public Void mergeAccumulators(Iterable accumulators) { + return null; + } + + @Override + public Void addInput(Void accumulator, Integer input) { + return accumulator; + } + + @Override + public boolean equals(Object other) { + return other != null && other.getClass().equals(TestCombineFn.class); + } + + @Override + public int hashCode() { + return TestCombineFn.class.hashCode(); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java index b405dd1aae30..ee24b3f1ef4b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java @@ -195,5 +195,15 @@ protected long getEncodedElementByteSize(long[] value) { } }; } + + @Override + public boolean equals(Object other) { + return other != null && getClass().equals(other.getClass()); + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java index ccade4db703a..6b6541606d20 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java @@ -151,6 +151,16 @@ public int apply(int a, int b) { public int identity() { return 0; } + + @Override + public boolean equals(Object other) { + return other != null && other.getClass().equals(this.getClass()); + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } } private static class SumLongFn extends Combine.BinaryCombineLongFn { @@ -164,6 +174,16 @@ public long apply(long a, long b) { public long identity() { return 0; } + + @Override + public boolean equals(Object other) { + return other != null && other.getClass().equals(this.getClass()); + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } } private static class SumDoubleFn extends Combine.BinaryCombineDoubleFn { @@ -177,5 +197,15 @@ public double apply(double a, double b) { public double identity() { return 0; } + + @Override + public boolean equals(Object other) { + return other != null && other.getClass().equals(this.getClass()); + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } } } From 9cd6c3bdefd50473f9fc6fac359213dcd6b4e4d4 Mon Sep 17 00:00:00 2001 From: MOLIG004 Date: Thu, 4 May 2017 09:21:23 -0700 Subject: [PATCH 067/159] Initial implementation of SpannerIO.Write This closes #2166. --- .../examples/spanner/SpannerCSVLoader.java | 143 +++++++++ pom.xml | 8 + sdks/java/io/google-cloud-platform/pom.xml | 33 ++- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 272 ++++++++++++++++++ .../beam/sdk/io/gcp/spanner/package-info.java | 23 ++ .../beam/sdk/io/gcp/GcpApiSurfaceTest.java | 2 + 6 files changed, 471 insertions(+), 10 deletions(-) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java new file mode 100644 index 000000000000..eee581da2148 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.spanner; + +import com.google.cloud.spanner.Database; +import com.google.cloud.spanner.DatabaseAdminClient; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Operation; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerException; +import com.google.cloud.spanner.SpannerOptions; +import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; +import java.util.Collections; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; +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.options.Validation; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; + + + +/** + * Generalized bulk loader for importing CSV files into Spanner. + * + */ +public class SpannerCSVLoader { + + /** + * Command options specification. + */ + private interface Options extends PipelineOptions { + @Description("Create a sample database") + @Default.Boolean(false) + boolean isCreateDatabase(); + void setCreateDatabase(boolean createDatabase); + + @Description("File to read from ") + @Validation.Required + String getInput(); + void setInput(String value); + + @Description("Instance ID to write to in Spanner") + @Validation.Required + String getInstanceId(); + void setInstanceId(String value); + + @Description("Database ID to write to in Spanner") + @Validation.Required + String getDatabaseId(); + void setDatabaseId(String value); + + @Description("Table name") + @Validation.Required + String getTable(); + void setTable(String value); + } + + + /** + * Constructs and executes the processing pipeline based upon command options. + */ + public static void main(String[] args) throws Exception { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + Pipeline p = Pipeline.create(options); + PCollection lines = p.apply(TextIO.Read.from(options.getInput())); + PCollection mutations = lines + .apply(ParDo.of(new NaiveParseCsvFn(options.getTable()))); + mutations + .apply(SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId())); + p.run().waitUntilFinish(); + } + + public static void createDatabase(Options options) { + Spanner client = SpannerOptions.getDefaultInstance().getService(); + + DatabaseAdminClient databaseAdminClient = client.getDatabaseAdminClient(); + try { + databaseAdminClient.dropDatabase(options.getInstanceId(), options + .getDatabaseId()); + } catch (SpannerException e) { + // Does not exist, ignore. + } + Operation op = databaseAdminClient.createDatabase( + options.getInstanceId(), options + .getDatabaseId(), Collections.singleton("CREATE TABLE " + options.getTable() + " (" + + " Key INT64," + + " Name STRING," + + " Email STRING," + + " Age INT," + + ") PRIMARY KEY (Key)")); + op.waitFor(); + } + + + /** + * A DoFn that creates a Spanner Mutation for each CSV line. + */ + static class NaiveParseCsvFn extends DoFn { + private final String table; + + NaiveParseCsvFn(String table) { + this.table = table; + } + + @ProcessElement + public void processElement(ProcessContext c) { + String line = c.element(); + String[] elements = line.split(","); + if (elements.length != 4) { + return; + } + Mutation mutation = Mutation.newInsertOrUpdateBuilder(table) + .set("Key").to(Long.valueOf(elements[0])) + .set("Name").to(elements[1]) + .set("Email").to(elements[2]) + .set("Age").to(Integer.valueOf(elements[3])) + .build(); + c.output(mutation); + } + } +} diff --git a/pom.xml b/pom.xml index cc2e4837a248..3a6289dfb7f2 100644 --- a/pom.xml +++ b/pom.xml @@ -104,6 +104,7 @@ 3.5 1.9 2.24.0 + 1.0.0-rc2 1.8.1 v2-rev295-1.22.0 0.9.6.2 @@ -139,6 +140,7 @@ 3.1.4 v1-rev71-1.22.0 4.4.1 + 0.16.0-beta 4.3.5.RELEASE 2.0 1.1.4-M3 @@ -865,6 +867,12 @@ ${google-cloud-bigdataoss.version} + + com.google.cloud + google-cloud-spanner + ${spanner.version} + + com.google.cloud.bigdataoss util diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index ea2d8f0cb290..21818959843f 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -81,11 +81,28 @@ jackson-databind + + io.grpc + grpc-core + + + + com.google.api.grpc + grpc-google-common-protos + ${grpc-google-common-protos.version} + + com.google.apis google-api-services-bigquery + + com.google.api + api-common + ${api-common.version} + + com.google.apis google-api-services-pubsub @@ -116,11 +133,6 @@ grpc-auth - - io.grpc - grpc-core - - io.grpc grpc-netty @@ -149,6 +161,12 @@ joda-time + + com.google.cloud + google-cloud-spanner + ${spanner.version} + + com.google.cloud.bigtable bigtable-protos @@ -184,11 +202,6 @@ google-auth-library-oauth2-http - - com.google.api.grpc - grpc-google-common-protos - - org.slf4j slf4j-api diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java new file mode 100644 index 000000000000..172ed8f02687 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -0,0 +1,272 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.gcp.spanner; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.Sleeper; + +import com.google.cloud.spanner.AbortedException; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerOptions; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.annotations.Experimental; +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.display.DisplayData; +import org.apache.beam.sdk.transforms.display.DisplayData.Builder; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + + + +/** + * Google Cloud Spanner connectors. + * + *

Reading from Cloud Spanner

+ * Status: Not implemented. + * + *

Writing to Cloud Spanner

+ * Status: Experimental. + * + *

{@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. + * To configure Cloud Spanner sink, you must apply {@link SpannerIO#writeTo} transform to + * {@link PCollection} and specify instance and database identifiers. + * For example, following code sketches out a pipeline that imports data from the CSV file to Cloud + * Spanner. + * + *

{@code
+ *
+ * Pipeline p = ...;
+ * // Read the CSV file.
+ * PCollection lines = p.apply("Read CSV file", TextIO.Read.from(options.getInput()));
+ * // Parse the line and convert to mutation.
+ * PCollection mutations = lines.apply("Parse CSV", parseFromCsv());
+ * // Write mutations.
+ * mutations.apply("Write", SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId()));
+ * p.run();
+ *
+ * }
+ + */ +@Experimental(Experimental.Kind.SOURCE_SINK) +public class SpannerIO { + + private SpannerIO() { + } + + @VisibleForTesting + static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; + + /** + * Creates an instance of {@link Writer}. Use {@link Writer#withBatchSize} to limit the batch + * size. + */ + public static Writer writeTo(String instanceId, String databaseId) { + return new Writer(instanceId, databaseId, SPANNER_MUTATIONS_PER_COMMIT_LIMIT); + } + + /** + * A {@link PTransform} that writes {@link Mutation} objects to Cloud Spanner. + * + * @see SpannerIO + */ + public static class Writer extends PTransform, PDone> { + + private final String instanceId; + private final String databaseId; + private int batchSize; + + Writer(String instanceId, String databaseId, int batchSize) { + this.instanceId = instanceId; + this.databaseId = databaseId; + this.batchSize = batchSize; + } + + /** + * Returns a new {@link Writer} with a limit on the number of mutations per batch. + * Defaults to {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + */ + public Writer withBatchSize(Integer batchSize) { + return new Writer(instanceId, databaseId, batchSize); + } + + @Override + public PDone expand(PCollection input) { + input.apply("Write mutations to Spanner", ParDo.of( + new SpannerWriterFn(instanceId, databaseId, batchSize))); + + return PDone.in(input.getPipeline()); + } + + @Override + public void validate(PCollection input) { + checkNotNull(instanceId, "instanceId"); + checkNotNull(databaseId, "databaseId"); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(getClass()) + .add("instanceId", instanceId) + .add("databaseId", databaseId) + .toString(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("instanceId", instanceId) + .withLabel("Output Instance")) + .addIfNotNull(DisplayData.item("databaseId", databaseId) + .withLabel("Output Database")); + } + + } + + + /** + * {@link DoFn} that writes {@link Mutation}s to Cloud Spanner. Mutations are written in + * batches, where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + * + *

See + * + *

Commits are non-transactional. If a commit fails, it will be retried (up to + * {@link SpannerIO#MAX_RETRIES}. times). This means that the + * mutation operation should be idempotent. + */ + @VisibleForTesting + static class SpannerWriterFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); + private transient Spanner spanner; + private final String instanceId; + private final String databaseId; + private final int batchSize; + private transient DatabaseClient dbClient; + // Current batch of mutations to be written. + private final List mutations = new ArrayList<>(); + + private static final int MAX_RETRIES = 5; + private static final FluentBackoff BUNDLE_WRITE_BACKOFF = + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); + + @VisibleForTesting + SpannerWriterFn(String instanceId, String databaseId, int batchSize) { + this.instanceId = checkNotNull(instanceId, "instanceId"); + this.databaseId = checkNotNull(databaseId, "databaseId"); + this.batchSize = batchSize; + } + + @Setup + public void setup() throws Exception { + SpannerOptions options = SpannerOptions.newBuilder().build(); + spanner = options.getService(); + dbClient = spanner.getDatabaseClient( + DatabaseId.of(options.getProjectId(), instanceId, databaseId)); + } + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + Mutation m = c.element(); + mutations.add(m); + int columnCount = m.asMap().size(); + if ((mutations.size() + 1) * columnCount >= batchSize) { + flushBatch(); + } + } + + @FinishBundle + public void finishBundle(Context c) throws Exception { + if (!mutations.isEmpty()) { + flushBatch(); + } + } + + @Teardown + public void teardown() throws Exception { + if (spanner == null) { + return; + } + spanner.closeAsync().get(); + } + + /** + * Writes a batch of mutations to Cloud Spanner. + * + *

If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. + * If the retry limit is exceeded, the last exception from Cloud Spanner will be + * thrown. + * + * @throws AbortedException if the commit fails or IOException or InterruptedException if + * backing off between retries fails. + */ + private void flushBatch() throws AbortedException, IOException, InterruptedException { + LOG.debug("Writing batch of {} mutations", mutations.size()); + Sleeper sleeper = Sleeper.DEFAULT; + BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff(); + + while (true) { + // Batch upsert rows. + try { + dbClient.writeAtLeastOnce(mutations); + + // Break if the commit threw no exception. + break; + } catch (AbortedException exception) { + // Only log the code and message for potentially-transient errors. The entire exception + // will be propagated upon the last retry. + LOG.error("Error writing to Spanner ({}): {}", exception.getCode(), + exception.getMessage()); + if (!BackOffUtils.next(sleeper, backoff)) { + LOG.error("Aborting after {} retries.", MAX_RETRIES); + throw exception; + } + } + } + LOG.debug("Successfully wrote {} mutations", mutations.size()); + mutations.clear(); + } + + @Override + public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("instanceId", instanceId) + .withLabel("Instance")) + .addIfNotNull(DisplayData.item("databaseId", databaseId) + .withLabel("Database")); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java new file mode 100644 index 000000000000..19e468cce041 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + *

Provides an API for reading from and writing to + * Google Cloud Spanner. + */ +package org.apache.beam.sdk.io.gcp.spanner; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java index 7025004cc953..8950452b5db2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java @@ -75,6 +75,8 @@ public void testGcpApiSurface() throws Exception { classesInPackage("javax"), classesInPackage("org.apache.beam"), classesInPackage("org.apache.commons.logging"), + classesInPackage("com.google.cloud"), + classesInPackage("com.google.cloud.spanner"), // via Bigtable classesInPackage("org.joda.time")); From e53dc915d8da30aa49fb28a632a5d890723a73dc Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 10:42:07 -0700 Subject: [PATCH 068/159] Minor style, compilation, javadoc fixups --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 45 +++++++++---------- 1 file changed, 20 insertions(+), 25 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 172ed8f02687..c9c81a51a384 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -19,10 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -import com.google.api.client.util.BackOff; -import com.google.api.client.util.BackOffUtils; -import com.google.api.client.util.Sleeper; - import com.google.cloud.spanner.AbortedException; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; @@ -35,23 +31,25 @@ import java.util.ArrayList; import java.util.List; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.PipelineOptions; 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.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.BackOffUtils; import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.Sleeper; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - - - /** - * Google Cloud Spanner connectors. + * {@link PTransform Transforms} for reading from and writing to + * Google Cloud Spanner. * *

Reading from Cloud Spanner

* Status: Not implemented. @@ -61,9 +59,8 @@ * *

{@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. * To configure Cloud Spanner sink, you must apply {@link SpannerIO#writeTo} transform to - * {@link PCollection} and specify instance and database identifiers. - * For example, following code sketches out a pipeline that imports data from the CSV file to Cloud - * Spanner. + * {@link PCollection} and specify instance and database identifiers. For example, following code + * sketches out a pipeline that imports data from the CSV file to Cloud Spanner. * *

{@code
  *
@@ -77,14 +74,10 @@
  * p.run();
  *
  * }
- */ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { - private SpannerIO() { - } - @VisibleForTesting static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; @@ -97,10 +90,11 @@ public static Writer writeTo(String instanceId, String databaseId) { } /** - * A {@link PTransform} that writes {@link Mutation} objects to Cloud Spanner. + * A {@link PTransform} that writes {@link Mutation} objects to Google Cloud Spanner. * * @see SpannerIO */ + @Experimental(Experimental.Kind.SOURCE_SINK) public static class Writer extends PTransform, PDone> { private final String instanceId; @@ -130,7 +124,7 @@ public PDone expand(PCollection input) { } @Override - public void validate(PCollection input) { + public void validate(PipelineOptions options) { checkNotNull(instanceId, "instanceId"); checkNotNull(databaseId, "databaseId"); } @@ -152,19 +146,17 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotNull(DisplayData.item("databaseId", databaseId) .withLabel("Output Database")); } - } - /** - * {@link DoFn} that writes {@link Mutation}s to Cloud Spanner. Mutations are written in + * {@link DoFn} that writes {@link Mutation}s to Google Cloud Spanner. Mutations are written in * batches, where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. * - *

See - * *

Commits are non-transactional. If a commit fails, it will be retried (up to - * {@link SpannerIO#MAX_RETRIES}. times). This means that the - * mutation operation should be idempotent. + * {@link SpannerWriterFn#MAX_RETRIES} times). This means that the mutation operation should be + * idempotent. + * + *

See Google Cloud Spanner documentation. */ @VisibleForTesting static class SpannerWriterFn extends DoFn { @@ -208,7 +200,7 @@ public void processElement(ProcessContext c) throws Exception { } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle() throws Exception { if (!mutations.isEmpty()) { flushBatch(); } @@ -269,4 +261,7 @@ public void populateDisplayData(Builder builder) { .withLabel("Database")); } } + + private SpannerIO() {} // Prevent construction. + } From 8871eadcc8413dab906656806b54f3d73b24893d Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 10:48:54 -0700 Subject: [PATCH 069/159] Fix spanner dependency management Also minor cleanup alphabetization in root pom.xml --- pom.xml | 22 +++++++++++++++++++--- sdks/java/io/google-cloud-platform/pom.xml | 8 +++++--- 2 files changed, 24 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 3a6289dfb7f2..c3a6b73dfb1a 100644 --- a/pom.xml +++ b/pom.xml @@ -115,11 +115,13 @@ 0.5.160222 1.4.0 1.3.0 + 1.0.0-rc2 1.0-rc2 1.4.1 0.6.1 1.22.0 1.4.5 + 1.0.2 0.5.160304 20.0 1.2.0 @@ -132,18 +134,20 @@ 1.9.5 4.1.8.Final 1.1.33.Fork26 - 1.5.0.Final 3.2.0 v1-rev10-1.22.0 1.7.14 + 0.16.0-beta 1.6.2 + 4.3.5.RELEASE 3.1.4 v1-rev71-1.22.0 4.4.1 - 0.16.0-beta 4.3.5.RELEASE - 2.0 1.1.4-M3 + + 1.5.0.Final + 2.0 2.20 2.20 3.6.1 @@ -610,6 +614,12 @@ ${grpc.version} + + com.google.api + api-common + ${google-api-common.version} + + com.google.api-client google-api-client @@ -867,6 +877,12 @@ ${google-cloud-bigdataoss.version} + + com.google.cloud + google-cloud-core + ${google-cloud-core.version} + + com.google.cloud google-cloud-spanner diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 21818959843f..9143ccf553cf 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -89,7 +89,6 @@ com.google.api.grpc grpc-google-common-protos - ${grpc-google-common-protos.version} @@ -100,7 +99,6 @@ com.google.api api-common - ${api-common.version} @@ -161,10 +159,14 @@ joda-time + + com.google.cloud + google-cloud-core + + com.google.cloud google-cloud-spanner - ${spanner.version} From 216b972b6b09c0bb612d7f716cfec68ace9777ec Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 10:48:24 -0700 Subject: [PATCH 070/159] Delete SpannerCSVLoader This is not appropriate for examples. SpannerIO should be well-javadoced and integration tested. --- .../examples/spanner/SpannerCSVLoader.java | 143 ------------------ 1 file changed, 143 deletions(-) delete mode 100644 examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java b/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java deleted file mode 100644 index eee581da2148..000000000000 --- a/examples/java/src/main/java/org/apache/beam/examples/spanner/SpannerCSVLoader.java +++ /dev/null @@ -1,143 +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.examples.spanner; - -import com.google.cloud.spanner.Database; -import com.google.cloud.spanner.DatabaseAdminClient; -import com.google.cloud.spanner.Mutation; -import com.google.cloud.spanner.Operation; -import com.google.cloud.spanner.Spanner; -import com.google.cloud.spanner.SpannerException; -import com.google.cloud.spanner.SpannerOptions; -import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; -import java.util.Collections; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.io.gcp.spanner.SpannerIO; -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.options.Validation; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; - - - -/** - * Generalized bulk loader for importing CSV files into Spanner. - * - */ -public class SpannerCSVLoader { - - /** - * Command options specification. - */ - private interface Options extends PipelineOptions { - @Description("Create a sample database") - @Default.Boolean(false) - boolean isCreateDatabase(); - void setCreateDatabase(boolean createDatabase); - - @Description("File to read from ") - @Validation.Required - String getInput(); - void setInput(String value); - - @Description("Instance ID to write to in Spanner") - @Validation.Required - String getInstanceId(); - void setInstanceId(String value); - - @Description("Database ID to write to in Spanner") - @Validation.Required - String getDatabaseId(); - void setDatabaseId(String value); - - @Description("Table name") - @Validation.Required - String getTable(); - void setTable(String value); - } - - - /** - * Constructs and executes the processing pipeline based upon command options. - */ - public static void main(String[] args) throws Exception { - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - - Pipeline p = Pipeline.create(options); - PCollection lines = p.apply(TextIO.Read.from(options.getInput())); - PCollection mutations = lines - .apply(ParDo.of(new NaiveParseCsvFn(options.getTable()))); - mutations - .apply(SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId())); - p.run().waitUntilFinish(); - } - - public static void createDatabase(Options options) { - Spanner client = SpannerOptions.getDefaultInstance().getService(); - - DatabaseAdminClient databaseAdminClient = client.getDatabaseAdminClient(); - try { - databaseAdminClient.dropDatabase(options.getInstanceId(), options - .getDatabaseId()); - } catch (SpannerException e) { - // Does not exist, ignore. - } - Operation op = databaseAdminClient.createDatabase( - options.getInstanceId(), options - .getDatabaseId(), Collections.singleton("CREATE TABLE " + options.getTable() + " (" - + " Key INT64," - + " Name STRING," - + " Email STRING," - + " Age INT," - + ") PRIMARY KEY (Key)")); - op.waitFor(); - } - - - /** - * A DoFn that creates a Spanner Mutation for each CSV line. - */ - static class NaiveParseCsvFn extends DoFn { - private final String table; - - NaiveParseCsvFn(String table) { - this.table = table; - } - - @ProcessElement - public void processElement(ProcessContext c) { - String line = c.element(); - String[] elements = line.split(","); - if (elements.length != 4) { - return; - } - Mutation mutation = Mutation.newInsertOrUpdateBuilder(table) - .set("Key").to(Long.valueOf(elements[0])) - .set("Name").to(elements[1]) - .set("Email").to(elements[2]) - .set("Age").to(Integer.valueOf(elements[3])) - .build(); - c.output(mutation); - } - } -} From 65e740ed2f265b550e70af867535d7bba7306298 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Mon, 15 May 2017 10:16:18 -0700 Subject: [PATCH 071/159] Refine Spanner API tests And remove outdated Bigtable comment --- .../org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java index 8950452b5db2..91caded1ad35 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java @@ -63,7 +63,10 @@ public void testGcpApiSurface() throws Exception { Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableClusterName.class), Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableInstanceName.class), Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableTableName.class), - // via Bigtable, PR above out to fix. + Matchers.>equalTo(com.google.cloud.ByteArray.class), + Matchers.>equalTo(com.google.cloud.Date.class), + Matchers.>equalTo(com.google.cloud.Timestamp.class), + classesInPackage("com.google.cloud.spanner"), classesInPackage("com.google.datastore.v1"), classesInPackage("com.google.protobuf"), classesInPackage("com.google.type"), @@ -75,9 +78,6 @@ public void testGcpApiSurface() throws Exception { classesInPackage("javax"), classesInPackage("org.apache.beam"), classesInPackage("org.apache.commons.logging"), - classesInPackage("com.google.cloud"), - classesInPackage("com.google.cloud.spanner"), - // via Bigtable classesInPackage("org.joda.time")); assertThat(apiSurface, containsOnlyClassesMatching(allowedClasses)); From e2070229f82ac4b78c470051cb94b68b58b75f16 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Mon, 15 May 2017 10:42:57 -0700 Subject: [PATCH 072/159] SpannerIO.Write cleanup and style fixes * Rename to Write to match the rest of the SDK. * Convert to AutoValue, delete toString. * Drop .writeTo(), instead use .write() as default constructor. * Temporarily drop withBatchSize, as its existence is not clearly justified. --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 116 ++++++++++-------- 1 file changed, 62 insertions(+), 54 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index c9c81a51a384..ec119311c106 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -19,6 +19,7 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.auto.value.AutoValue; import com.google.cloud.spanner.AbortedException; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; @@ -26,12 +27,11 @@ import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerOptions; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.MoreObjects; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -48,31 +48,29 @@ import org.slf4j.LoggerFactory; /** - * {@link PTransform Transforms} for reading from and writing to + * Experimental {@link PTransform Transforms} for reading from and writing to * Google Cloud Spanner. * *

Reading from Cloud Spanner

- * Status: Not implemented. + * + *

This functionality is not yet implemented. * *

Writing to Cloud Spanner

- * Status: Experimental. * - *

{@link SpannerIO#writeTo} batches together and concurrently writes a set of {@link Mutation}s. - * To configure Cloud Spanner sink, you must apply {@link SpannerIO#writeTo} transform to - * {@link PCollection} and specify instance and database identifiers. For example, following code - * sketches out a pipeline that imports data from the CSV file to Cloud Spanner. + *

The Cloud Spanner {@link SpannerIO.Write} transform writes to Cloud Spanner by executing a + * collection of input row {@link Mutation Mutations}. The mutations grouped into batches for + * efficiency. * - *

{@code
+ * 

To configure the write transform, create an instance using {@link #write()} and then specify + * the destination Cloud Spanner instance ({@link Write#withInstanceId(String)} and destination + * database ({@link Write#withDatabaseId(String)}). For example: * - * Pipeline p = ...; - * // Read the CSV file. - * PCollection lines = p.apply("Read CSV file", TextIO.Read.from(options.getInput())); - * // Parse the line and convert to mutation. - * PCollection mutations = lines.apply("Parse CSV", parseFromCsv()); + *

{@code
+ * // Earlier in the pipeline, create a PCollection of Mutations to be written to Cloud Spanner.
+ * PCollection mutations = ...;
  * // Write mutations.
- * mutations.apply("Write", SpannerIO.writeTo(options.getInstanceId(), options.getDatabaseId()));
- * p.run();
- *
+ * mutations.apply(
+ *     "Write", SpannerIO.write().withInstanceId("instance").withDatabaseId("database"));
  * }
*/ @Experimental(Experimental.Kind.SOURCE_SINK) @@ -81,12 +79,14 @@ public class SpannerIO { @VisibleForTesting static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; - /** - * Creates an instance of {@link Writer}. Use {@link Writer#withBatchSize} to limit the batch - * size. - */ - public static Writer writeTo(String instanceId, String databaseId) { - return new Writer(instanceId, databaseId, SPANNER_MUTATIONS_PER_COMMIT_LIMIT); + /** + * Creates an unitialized instance of {@link Write}. Before use, the {@link Write} must be + * configured with a {@link Write#withInstanceId} and {@link Write#withDatabaseId} that identify + * the Cloud Spanner database being written. + */ + @Experimental + public static Write write() { + return new AutoValue_SpannerIO_Write.Builder().build(); } /** @@ -95,55 +95,63 @@ public static Writer writeTo(String instanceId, String databaseId) { * @see SpannerIO */ @Experimental(Experimental.Kind.SOURCE_SINK) - public static class Writer extends PTransform, PDone> { + @AutoValue + public abstract static class Write extends PTransform, PDone> { - private final String instanceId; - private final String databaseId; - private int batchSize; + @Nullable + abstract String getInstanceId(); - Writer(String instanceId, String databaseId, int batchSize) { - this.instanceId = instanceId; - this.databaseId = databaseId; - this.batchSize = batchSize; + @Nullable + abstract String getDatabaseId(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setInstanceId(String instanceId); + + abstract Builder setDatabaseId(String databaseId); + + abstract Write build(); } /** - * Returns a new {@link Writer} with a limit on the number of mutations per batch. - * Defaults to {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. + * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner + * instance. + * + *

Does not modify this object. */ - public Writer withBatchSize(Integer batchSize) { - return new Writer(instanceId, databaseId, batchSize); + public Write withInstanceId(String instanceId) { + return toBuilder().setInstanceId(instanceId).build(); } - @Override - public PDone expand(PCollection input) { - input.apply("Write mutations to Spanner", ParDo.of( - new SpannerWriterFn(instanceId, databaseId, batchSize))); - - return PDone.in(input.getPipeline()); + /** + * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner + * database. + * + *

Does not modify this object. + */ + public Write withDatabaseId(String databaseId) { + return toBuilder().setDatabaseId(databaseId).build(); } @Override - public void validate(PipelineOptions options) { - checkNotNull(instanceId, "instanceId"); - checkNotNull(databaseId, "databaseId"); - } + public PDone expand(PCollection input) { + input.apply("Write mutations to Spanner", + ParDo.of(new SpannerWriterFn( + getInstanceId(), getDatabaseId(), SPANNER_MUTATIONS_PER_COMMIT_LIMIT))); - @Override - public String toString() { - return MoreObjects.toStringHelper(getClass()) - .add("instanceId", instanceId) - .add("databaseId", databaseId) - .toString(); + return PDone.in(input.getPipeline()); } @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("instanceId", instanceId) + .addIfNotNull(DisplayData.item("instanceId", getInstanceId()) .withLabel("Output Instance")) - .addIfNotNull(DisplayData.item("databaseId", databaseId) + .addIfNotNull(DisplayData.item("databaseId", getDatabaseId()) .withLabel("Output Database")); } } From bb2b7064960c458a63c627d96e948f38ef11a898 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 23 May 2017 12:28:59 -0700 Subject: [PATCH 073/159] Automatically generate Python proto and grpc files. --- .gitignore | 1 + pom.xml | 2 +- sdks/python/MANIFEST.in | 1 + .../apache_beam/runners/api/__init__.py | 13 +- sdks/python/gen_protos.py | 116 ++++++++++++++++++ sdks/python/run_pylint.sh | 1 + sdks/python/setup.py | 38 +++++- sdks/python/tox.ini | 3 + 8 files changed, 157 insertions(+), 18 deletions(-) create mode 100644 sdks/python/gen_protos.py diff --git a/.gitignore b/.gitignore index 1ecb993777b0..6e077bd1d440 100644 --- a/.gitignore +++ b/.gitignore @@ -25,6 +25,7 @@ sdks/python/**/*.egg sdks/python/LICENSE sdks/python/NOTICE sdks/python/README.md +sdks/python/runners/api/*pb2*.* # Ignore IntelliJ files. .idea/ diff --git a/pom.xml b/pom.xml index c3a6b73dfb1a..4f833cdb8e49 100644 --- a/pom.xml +++ b/pom.xml @@ -1301,7 +1301,7 @@ **/.settings/**/* - **/sdks/python/apache_beam/runners/api/*.py + **/apache_beam/runners/api/*_pb2*.py diff --git a/sdks/python/MANIFEST.in b/sdks/python/MANIFEST.in index 41d80ef1b7ec..c97e57aa1a16 100644 --- a/sdks/python/MANIFEST.in +++ b/sdks/python/MANIFEST.in @@ -15,6 +15,7 @@ # limitations under the License. # +include gen_protos.py include README.md include NOTICE include LICENSE diff --git a/sdks/python/apache_beam/runners/api/__init__.py b/sdks/python/apache_beam/runners/api/__init__.py index bf95208bd6f9..2750859e4f66 100644 --- a/sdks/python/apache_beam/runners/api/__init__.py +++ b/sdks/python/apache_beam/runners/api/__init__.py @@ -17,16 +17,5 @@ """For internal use only; no backwards-compatibility guarantees. -Checked in to avoid protoc dependency for Python development. - -Regenerate files with:: - - protoc -I../common/runner-api/src/main/proto/ \ - --python_out=apache_beam/runners/api/ \ - ../common/runner-api/src/main/proto/*.proto - - protoc -I../common/{fn,runner}-api/src/main/proto/ \ - --python_out=apache_beam/runners/api/ \ - --grpc_python_out=apache_beam/runners/api/ \ - ../common/fn-api/src/main/proto/*.proto +Automatically generated when running setup.py sdist or build[_py]. """ diff --git a/sdks/python/gen_protos.py b/sdks/python/gen_protos.py new file mode 100644 index 000000000000..edaaff488dfb --- /dev/null +++ b/sdks/python/gen_protos.py @@ -0,0 +1,116 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""Generates Python proto modules and grpc stubs for Beam protos.""" + +import glob +import logging +import multiprocessing +import os +import pkg_resources +import subprocess +import sys +import warnings + + +GRPC_TOOLS = 'grpcio-tools>=1.3.5' + +BEAM_PROTO_PATHS = [ + os.path.join('..', 'common', 'runner-api', 'src', 'main', 'proto'), + os.path.join('..', 'common', 'fn-api', 'src', 'main', 'proto') +] + +PYTHON_OUTPUT_PATH = os.path.join('apache_beam', 'runners', 'api') + + +def generate_proto_files(): + + try: + import grpc_tools + except ImportError: + warnings.warn('Installing grpcio-tools is recommended for development.') + + py_sdk_root = os.path.dirname(os.path.abspath(__file__)) + common = os.path.join(py_sdk_root, '..', 'common') + proto_dirs = [os.path.join(py_sdk_root, path) for path in BEAM_PROTO_PATHS] + proto_files = sum( + [glob.glob(os.path.join(d, '*.proto')) for d in proto_dirs], []) + out_dir = os.path.join(py_sdk_root, PYTHON_OUTPUT_PATH) + out_files = [path for path in glob.glob(os.path.join(out_dir, '*_pb2.py'))] + + if out_files and not proto_files: + # We have out_files but no protos; assume they're up to date. + # This is actually the common case (e.g. installation from an sdist). + return + + elif not out_files and not proto_files: + if not common: + raise RuntimeError( + 'Not in apache git tree; unable to find proto definitions.') + else: + raise RuntimeError( + 'No proto files found in %s.' % proto_dirs) + + # Regenerate iff the proto files are newer. + elif not out_files or len(out_files) < len(proto_files) or ( + min(os.path.getmtime(path) for path in out_files) + <= max(os.path.getmtime(path) for path in proto_files)): + try: + from grpc_tools import protoc + except ImportError: + # Use a subprocess to avoid messing with this process' path and imports. + # Note that this requires a separate module from setup.py for Windows: + # https://docs.python.org/2/library/multiprocessing.html#windows + p = multiprocessing.Process( + target=_install_grpcio_tools_and_generate_proto_files) + p.start() + p.join() + else: + logging.info('Regenerating out-of-date Python proto definitions.') + builtin_protos = pkg_resources.resource_filename('grpc_tools', '_proto') + args = ( + [sys.executable] + # expecting to be called from command line + ['--proto_path=%s' % builtin_protos] + + ['--proto_path=%s' % d for d in proto_dirs] + + ['--python_out=%s' % out_dir] + + ['--grpc_python_out=%s' % out_dir] + + proto_files) + ret_code = protoc.main(args) + if ret_code: + raise RuntimeError( + 'Protoc returned non-zero status (see logs for details): ' + '%s' % ret_code) + + +# Though wheels are available for grpcio-tools, setup_requires uses +# easy_install which doesn't understand them. This means that it is +# compiled from scratch (which is expensive as it compiles the full +# protoc compiler). Instead, we attempt to install a wheel in a temporary +# directory and add it to the path as needed. +# See https://github.com/pypa/setuptools/issues/377 +def _install_grpcio_tools_and_generate_proto_files(): + install_path = os.path.join( + os.path.dirname(os.path.abspath(__file__)), '.eggs', 'grpcio-wheels') + logging.warning('Downloading a grpcio-tools to %s' % install_path) + subprocess.check_call( + ['pip', 'install', '-t', install_path, '--upgrade', GRPC_TOOLS]) + sys.path.append(install_path) + generate_proto_files() + + +if __name__ == '__main__': + generate_proto_files() \ No newline at end of file diff --git a/sdks/python/run_pylint.sh b/sdks/python/run_pylint.sh index a5e3fa147a81..400c577519dd 100755 --- a/sdks/python/run_pylint.sh +++ b/sdks/python/run_pylint.sh @@ -49,6 +49,7 @@ EXCLUDED_GENERATED_FILES=( "apache_beam/runners/api/beam_fn_api_pb2.py" "apache_beam/runners/api/beam_fn_api_pb2_grpc.py" "apache_beam/runners/api/beam_runner_api_pb2.py" +"apache_beam/runners/api/beam_runner_api_pb2_grpc.py" ) FILES_TO_IGNORE="" diff --git a/sdks/python/setup.py b/sdks/python/setup.py index befc0249c0bc..8a8ce482742a 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -19,15 +19,25 @@ from distutils.version import StrictVersion +import glob import os +import pkg_resources import platform import shutil +import subprocess +import sys import warnings import setuptools +from setuptools.command.build_py import build_py +from setuptools.command.sdist import sdist +from setuptools.command.test import test + from pkg_resources import get_distribution, DistributionNotFound +import gen_protos + def get_version(): global_names = {} @@ -98,8 +108,12 @@ def get_version(): 'pyyaml>=3.12,<4.0.0', ] +REQUIRED_SETUP_PACKAGES = [ + 'nose>=1.0', + ] + REQUIRED_TEST_PACKAGES = [ - 'pyhamcrest>=1.9,<2.0' + 'pyhamcrest>=1.9,<2.0', ] GCP_REQUIREMENTS = [ @@ -111,6 +125,15 @@ def get_version(): ] +# We must generate protos after setup_requires are installed. +def generate_protos_first(original_cmd): + class cmd(original_cmd, object): + def run(self): + gen_protos.generate_proto_files() + super(cmd, self).run() + return cmd + + setuptools.setup( name=PACKAGE_NAME, version=PACKAGE_VERSION, @@ -135,7 +158,7 @@ def get_version(): 'apache_beam/utils/counters.py', 'apache_beam/utils/windowed_value.py', ]), - setup_requires=['nose>=1.0'], + setup_requires=REQUIRED_SETUP_PACKAGES, install_requires=REQUIRED_PACKAGES, test_suite='nose.collector', tests_require=REQUIRED_TEST_PACKAGES, @@ -153,11 +176,16 @@ def get_version(): 'Programming Language :: Python :: 2.7', 'Topic :: Software Development :: Libraries', 'Topic :: Software Development :: Libraries :: Python Modules', - ], + ], license='Apache License, Version 2.0', keywords=PACKAGE_KEYWORDS, entry_points={ 'nose.plugins.0.10': [ 'beam_test_plugin = test_config:BeamTestPlugin' - ]} - ) + ]}, + cmdclass={ + 'build_py': generate_protos_first(build_py), + 'sdist': generate_protos_first(sdist), + 'test': generate_protos_first(test), + }, +) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 2592b17501ae..917e907b5ce2 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -28,6 +28,7 @@ select = E3 # autocomplete_test depends on nose when invoked directly. deps = nose==1.3.7 + grpcio-tools==1.3.5 commands = python --version pip install -e .[test] @@ -44,6 +45,7 @@ platform = linux2 # autocomplete_test depends on nose when invoked directly. deps = nose==1.3.7 + grpcio-tools==1.3.5 cython==0.25.2 whitelist_externals=find commands = @@ -87,6 +89,7 @@ passenv = TRAVIS* [testenv:docs] deps= nose==1.3.7 + grpcio-tools==1.3.5 Sphinx==1.5.5 commands = pip install -e .[test,gcp,docs] From c2e85daa68b444a9daeb582e686400bdee405964 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 23 May 2017 12:30:30 -0700 Subject: [PATCH 074/159] Remove auto-generated proto and grpc files. --- .../runners/api/beam_fn_api_pb2.py | 2742 ---------------- .../runners/api/beam_fn_api_pb2_grpc.py | 205 -- .../runners/api/beam_runner_api_pb2.py | 2872 ----------------- 3 files changed, 5819 deletions(-) delete mode 100644 sdks/python/apache_beam/runners/api/beam_fn_api_pb2.py delete mode 100644 sdks/python/apache_beam/runners/api/beam_fn_api_pb2_grpc.py delete mode 100644 sdks/python/apache_beam/runners/api/beam_runner_api_pb2.py diff --git a/sdks/python/apache_beam/runners/api/beam_fn_api_pb2.py b/sdks/python/apache_beam/runners/api/beam_fn_api_pb2.py deleted file mode 100644 index cb0b72b7d1fc..000000000000 --- a/sdks/python/apache_beam/runners/api/beam_fn_api_pb2.py +++ /dev/null @@ -1,2742 +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. -# - -# Generated by the protocol buffer compiler. DO NOT EDIT! -# source: beam_fn_api.proto - -import sys -_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) -from google.protobuf import descriptor as _descriptor -from google.protobuf import message as _message -from google.protobuf import reflection as _reflection -from google.protobuf import symbol_database as _symbol_database -from google.protobuf import descriptor_pb2 -# @@protoc_insertion_point(imports) - -_sym_db = _symbol_database.Default() - - -from google.protobuf import any_pb2 as google_dot_protobuf_dot_any__pb2 -from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 - -# This module is experimental. No backwards-compatibility guarantees. - - -DESCRIPTOR = _descriptor.FileDescriptor( - name='beam_fn_api.proto', - package='org.apache.beam.fn.v1', - syntax='proto3', - serialized_pb=_b('\n\x11\x62\x65\x61m_fn_api.proto\x12\x15org.apache.beam.fn.v1\x1a\x19google/protobuf/any.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"t\n\x06Target\x12%\n\x1dprimitive_transform_reference\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x1a\x35\n\x04List\x12-\n\x06target\x18\x01 \x03(\x0b\x32\x1d.org.apache.beam.fn.v1.Target\"&\n\x0bPCollection\x12\x17\n\x0f\x63oder_reference\x18\x01 \x01(\t\"\xcb\x04\n\x12PrimitiveTransform\x12\n\n\x02id\x18\x01 \x01(\t\x12:\n\rfunction_spec\x18\x02 \x01(\x0b\x32#.org.apache.beam.fn.v1.FunctionSpec\x12\x45\n\x06inputs\x18\x03 \x03(\x0b\x32\x35.org.apache.beam.fn.v1.PrimitiveTransform.InputsEntry\x12G\n\x07outputs\x18\x04 \x03(\x0b\x32\x36.org.apache.beam.fn.v1.PrimitiveTransform.OutputsEntry\x12N\n\x0bside_inputs\x18\x05 \x03(\x0b\x32\x39.org.apache.beam.fn.v1.PrimitiveTransform.SideInputsEntry\x12\x11\n\tstep_name\x18\x06 \x01(\t\x1aQ\n\x0bInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x31\n\x05value\x18\x02 \x01(\x0b\x32\".org.apache.beam.fn.v1.Target.List:\x02\x38\x01\x1aR\n\x0cOutputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x31\n\x05value\x18\x02 \x01(\x0b\x32\".org.apache.beam.fn.v1.PCollection:\x02\x38\x01\x1aS\n\x0fSideInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12/\n\x05value\x18\x02 \x01(\x0b\x32 .org.apache.beam.fn.v1.SideInput:\x02\x38\x01\"j\n\x0c\x46unctionSpec\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0b\n\x03urn\x18\x02 \x01(\t\x12\x1d\n\x15\x65nvironment_reference\x18\x03 \x01(\t\x12\"\n\x04\x64\x61ta\x18\x04 \x01(\x0b\x32\x14.google.protobuf.Any\"o\n\tSideInput\x12,\n\x05input\x18\x01 \x01(\x0b\x32\x1d.org.apache.beam.fn.v1.Target\x12\x34\n\x07view_fn\x18\x02 \x01(\x0b\x32#.org.apache.beam.fn.v1.FunctionSpec\"f\n\x05\x43oder\x12:\n\rfunction_spec\x18\x01 \x01(\x0b\x32#.org.apache.beam.fn.v1.FunctionSpec\x12!\n\x19\x63omponent_coder_reference\x18\x02 \x03(\t\"]\n\x0eRemoteGrpcPort\x12K\n\x16\x61pi_service_descriptor\x18\x01 \x01(\x0b\x32+.org.apache.beam.fn.v1.ApiServiceDescriptor\"\xe8\x02\n\x12InstructionRequest\x12\x16\n\x0einstruction_id\x18\x01 \x01(\t\x12;\n\x08register\x18\xe8\x07 \x01(\x0b\x32&.org.apache.beam.fn.v1.RegisterRequestH\x00\x12\x46\n\x0eprocess_bundle\x18\xe9\x07 \x01(\x0b\x32+.org.apache.beam.fn.v1.ProcessBundleRequestH\x00\x12W\n\x17process_bundle_progress\x18\xea\x07 \x01(\x0b\x32\x33.org.apache.beam.fn.v1.ProcessBundleProgressRequestH\x00\x12Q\n\x14process_bundle_split\x18\xeb\x07 \x01(\x0b\x32\x30.org.apache.beam.fn.v1.ProcessBundleSplitRequestH\x00\x42\t\n\x07request\"\xfd\x02\n\x13InstructionResponse\x12\x16\n\x0einstruction_id\x18\x01 \x01(\t\x12\r\n\x05\x65rror\x18\x02 \x01(\t\x12<\n\x08register\x18\xe8\x07 \x01(\x0b\x32\'.org.apache.beam.fn.v1.RegisterResponseH\x00\x12G\n\x0eprocess_bundle\x18\xe9\x07 \x01(\x0b\x32,.org.apache.beam.fn.v1.ProcessBundleResponseH\x00\x12X\n\x17process_bundle_progress\x18\xea\x07 \x01(\x0b\x32\x34.org.apache.beam.fn.v1.ProcessBundleProgressResponseH\x00\x12R\n\x14process_bundle_split\x18\xeb\x07 \x01(\x0b\x32\x31.org.apache.beam.fn.v1.ProcessBundleSplitResponseH\x00\x42\n\n\x08response\"d\n\x0fRegisterRequest\x12Q\n\x19process_bundle_descriptor\x18\x01 \x03(\x0b\x32..org.apache.beam.fn.v1.ProcessBundleDescriptor\"\x12\n\x10RegisterResponse\"\x9b\x01\n\x17ProcessBundleDescriptor\x12\n\n\x02id\x18\x01 \x01(\t\x12\x46\n\x13primitive_transform\x18\x02 \x03(\x0b\x32).org.apache.beam.fn.v1.PrimitiveTransform\x12,\n\x06\x63oders\x18\x04 \x03(\x0b\x32\x1c.org.apache.beam.fn.v1.Coder\"|\n\x14ProcessBundleRequest\x12+\n#process_bundle_descriptor_reference\x18\x01 \x01(\t\x12\x37\n\x0c\x63\x61\x63he_tokens\x18\x02 \x03(\x0b\x32!.org.apache.beam.fn.v1.CacheToken\"\x17\n\x15ProcessBundleResponse\"=\n\x1cProcessBundleProgressRequest\x12\x1d\n\x15instruction_reference\x18\x01 \x01(\t\"G\n\x1dProcessBundleProgressResponse\x12\x15\n\rfinished_work\x18\x01 \x01(\x01\x12\x0f\n\x07\x62\x61\x63klog\x18\x02 \x01(\x01\"L\n\x19ProcessBundleSplitRequest\x12\x1d\n\x15instruction_reference\x18\x01 \x01(\t\x12\x10\n\x08\x66raction\x18\x02 \x01(\x01\"(\n\x17\x45lementCountRestriction\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\",\n\x1b\x45lementCountSkipRestriction\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\"\xc8\x01\n\x17PrimitiveTransformSplit\x12%\n\x1dprimitive_transform_reference\x18\x01 \x01(\t\x12\x42\n\x15\x63ompleted_restriction\x18\x02 \x01(\x0b\x32#.org.apache.beam.fn.v1.FunctionSpec\x12\x42\n\x15remaining_restriction\x18\x03 \x01(\x0b\x32#.org.apache.beam.fn.v1.FunctionSpec\"\\\n\x1aProcessBundleSplitResponse\x12>\n\x06splits\x18\x01 \x03(\x0b\x32..org.apache.beam.fn.v1.PrimitiveTransformSplit\"\xa2\x01\n\x08\x45lements\x12\x32\n\x04\x64\x61ta\x18\x01 \x03(\x0b\x32$.org.apache.beam.fn.v1.Elements.Data\x1a\x62\n\x04\x44\x61ta\x12\x1d\n\x15instruction_reference\x18\x01 \x01(\t\x12-\n\x06target\x18\x02 \x01(\x0b\x32\x1d.org.apache.beam.fn.v1.Target\x12\x0c\n\x04\x64\x61ta\x18\x03 \x01(\x0c\"\xaa\x02\n\x0cStateRequest\x12\n\n\x02id\x18\x01 \x01(\t\x12\x1d\n\x15instruction_reference\x18\x02 \x01(\t\x12\x32\n\tstate_key\x18\x03 \x01(\x0b\x32\x1f.org.apache.beam.fn.v1.StateKey\x12\x36\n\x03get\x18\xe8\x07 \x01(\x0b\x32&.org.apache.beam.fn.v1.StateGetRequestH\x00\x12<\n\x06\x61ppend\x18\xe9\x07 \x01(\x0b\x32).org.apache.beam.fn.v1.StateAppendRequestH\x00\x12:\n\x05\x63lear\x18\xea\x07 \x01(\x0b\x32(.org.apache.beam.fn.v1.StateClearRequestH\x00\x42\t\n\x07request\"\xeb\x01\n\rStateResponse\x12\n\n\x02id\x18\x01 \x01(\t\x12\r\n\x05\x65rror\x18\x02 \x01(\t\x12\x37\n\x03get\x18\xe8\x07 \x01(\x0b\x32\'.org.apache.beam.fn.v1.StateGetResponseH\x00\x12=\n\x06\x61ppend\x18\xe9\x07 \x01(\x0b\x32*.org.apache.beam.fn.v1.StateAppendResponseH\x00\x12;\n\x05\x63lear\x18\xea\x07 \x01(\x0b\x32).org.apache.beam.fn.v1.StateClearResponseH\x00\x42\n\n\x08response\"J\n\nCacheToken\x12-\n\x06target\x18\x01 \x01(\x0b\x32\x1d.org.apache.beam.fn.v1.Target\x12\r\n\x05token\x18\x02 \x01(\x0c\"V\n\x08StateKey\x12-\n\x06target\x18\x01 \x01(\x0b\x32\x1d.org.apache.beam.fn.v1.Target\x12\x0e\n\x06window\x18\x02 \x01(\x0c\x12\x0b\n\x03key\x18\x03 \x01(\x0c\"=\n\x11\x43ontinuableStream\x12\x1a\n\x12\x63ontinuation_token\x18\x01 \x01(\x0c\x12\x0c\n\x04\x64\x61ta\x18\x02 \x01(\x0c\"-\n\x0fStateGetRequest\x12\x1a\n\x12\x63ontinuation_token\x18\x01 \x01(\x0c\"L\n\x10StateGetResponse\x12\x38\n\x06stream\x18\x01 \x01(\x0b\x32(.org.apache.beam.fn.v1.ContinuableStream\"\"\n\x12StateAppendRequest\x12\x0c\n\x04\x64\x61ta\x18\x01 \x01(\x0c\"\x15\n\x13StateAppendResponse\"\x13\n\x11StateClearRequest\"\x14\n\x12StateClearResponse\"\x9a\x03\n\x08LogEntry\x12:\n\x08severity\x18\x01 \x01(\x0e\x32(.org.apache.beam.fn.v1.LogEntry.Severity\x12-\n\ttimestamp\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0f\n\x07message\x18\x03 \x01(\t\x12\r\n\x05trace\x18\x04 \x01(\t\x12\x1d\n\x15instruction_reference\x18\x05 \x01(\t\x12%\n\x1dprimitive_transform_reference\x18\x06 \x01(\t\x12\x14\n\x0clog_location\x18\x07 \x01(\t\x12\x0e\n\x06thread\x18\x08 \x01(\t\x1a<\n\x04List\x12\x34\n\x0blog_entries\x18\x01 \x03(\x0b\x32\x1f.org.apache.beam.fn.v1.LogEntry\"Y\n\x08Severity\x12\t\n\x05TRACE\x10\x00\x12\t\n\x05\x44\x45\x42UG\x10\n\x12\x08\n\x04INFO\x10\x14\x12\n\n\x06NOTICE\x10\x1e\x12\x08\n\x04WARN\x10(\x12\t\n\x05\x45RROR\x10\x32\x12\x0c\n\x08\x43RITICAL\x10<\"\x0c\n\nLogControl\"\xa1\x01\n\x14\x41piServiceDescriptor\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0b\n\x03url\x18\x02 \x01(\t\x12^\n\x1foauth2_client_credentials_grant\x18\x03 \x01(\x0b\x32\x33.org.apache.beam.fn.v1.OAuth2ClientCredentialsGrantH\x00\x42\x10\n\x0e\x61uthentication\"+\n\x1cOAuth2ClientCredentialsGrant\x12\x0b\n\x03url\x18\x01 \x01(\t\"F\n\x0f\x44ockerContainer\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0b\n\x03uri\x18\x02 \x01(\t\x12\x1a\n\x12registry_reference\x18\x03 \x01(\t2w\n\rBeamFnControl\x12\x66\n\x07\x43ontrol\x12*.org.apache.beam.fn.v1.InstructionResponse\x1a).org.apache.beam.fn.v1.InstructionRequest\"\x00(\x01\x30\x01\x32\\\n\nBeamFnData\x12N\n\x04\x44\x61ta\x12\x1f.org.apache.beam.fn.v1.Elements\x1a\x1f.org.apache.beam.fn.v1.Elements\"\x00(\x01\x30\x01\x32g\n\x0b\x42\x65\x61mFnState\x12X\n\x05State\x12#.org.apache.beam.fn.v1.StateRequest\x1a$.org.apache.beam.fn.v1.StateResponse\"\x00(\x01\x30\x01\x32i\n\rBeamFnLogging\x12X\n\x07Logging\x12$.org.apache.beam.fn.v1.LogEntry.List\x1a!.org.apache.beam.fn.v1.LogControl\"\x00(\x01\x30\x01\x42\"\n\x15org.apache.beam.fn.v1B\tBeamFnApib\x06proto3') - , - dependencies=[google_dot_protobuf_dot_any__pb2.DESCRIPTOR,google_dot_protobuf_dot_timestamp__pb2.DESCRIPTOR,]) -_sym_db.RegisterFileDescriptor(DESCRIPTOR) - - - -_LOGENTRY_SEVERITY = _descriptor.EnumDescriptor( - name='Severity', - full_name='org.apache.beam.fn.v1.LogEntry.Severity', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='TRACE', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='DEBUG', index=1, number=10, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='INFO', index=2, number=20, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='NOTICE', index=3, number=30, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='WARN', index=4, number=40, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='ERROR', index=5, number=50, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='CRITICAL', index=6, number=60, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=4529, - serialized_end=4618, -) -_sym_db.RegisterEnumDescriptor(_LOGENTRY_SEVERITY) - - -_TARGET_LIST = _descriptor.Descriptor( - name='List', - full_name='org.apache.beam.fn.v1.Target.List', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='target', full_name='org.apache.beam.fn.v1.Target.List.target', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=167, - serialized_end=220, -) - -_TARGET = _descriptor.Descriptor( - name='Target', - full_name='org.apache.beam.fn.v1.Target', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='primitive_transform_reference', full_name='org.apache.beam.fn.v1.Target.primitive_transform_reference', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='name', full_name='org.apache.beam.fn.v1.Target.name', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_TARGET_LIST, ], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=104, - serialized_end=220, -) - - -_PCOLLECTION = _descriptor.Descriptor( - name='PCollection', - full_name='org.apache.beam.fn.v1.PCollection', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='coder_reference', full_name='org.apache.beam.fn.v1.PCollection.coder_reference', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=222, - serialized_end=260, -) - - -_PRIMITIVETRANSFORM_INPUTSENTRY = _descriptor.Descriptor( - name='InputsEntry', - full_name='org.apache.beam.fn.v1.PrimitiveTransform.InputsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.fn.v1.PrimitiveTransform.InputsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.fn.v1.PrimitiveTransform.InputsEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=600, - serialized_end=681, -) - -_PRIMITIVETRANSFORM_OUTPUTSENTRY = _descriptor.Descriptor( - name='OutputsEntry', - full_name='org.apache.beam.fn.v1.PrimitiveTransform.OutputsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.fn.v1.PrimitiveTransform.OutputsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.fn.v1.PrimitiveTransform.OutputsEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=683, - serialized_end=765, -) - -_PRIMITIVETRANSFORM_SIDEINPUTSENTRY = _descriptor.Descriptor( - name='SideInputsEntry', - full_name='org.apache.beam.fn.v1.PrimitiveTransform.SideInputsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.fn.v1.PrimitiveTransform.SideInputsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.fn.v1.PrimitiveTransform.SideInputsEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=767, - serialized_end=850, -) - -_PRIMITIVETRANSFORM = _descriptor.Descriptor( - name='PrimitiveTransform', - full_name='org.apache.beam.fn.v1.PrimitiveTransform', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='id', full_name='org.apache.beam.fn.v1.PrimitiveTransform.id', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='function_spec', full_name='org.apache.beam.fn.v1.PrimitiveTransform.function_spec', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='inputs', full_name='org.apache.beam.fn.v1.PrimitiveTransform.inputs', index=2, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='outputs', full_name='org.apache.beam.fn.v1.PrimitiveTransform.outputs', index=3, - number=4, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='side_inputs', full_name='org.apache.beam.fn.v1.PrimitiveTransform.side_inputs', index=4, - number=5, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='step_name', full_name='org.apache.beam.fn.v1.PrimitiveTransform.step_name', index=5, - number=6, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_PRIMITIVETRANSFORM_INPUTSENTRY, _PRIMITIVETRANSFORM_OUTPUTSENTRY, _PRIMITIVETRANSFORM_SIDEINPUTSENTRY, ], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=263, - serialized_end=850, -) - - -_FUNCTIONSPEC = _descriptor.Descriptor( - name='FunctionSpec', - full_name='org.apache.beam.fn.v1.FunctionSpec', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='id', full_name='org.apache.beam.fn.v1.FunctionSpec.id', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='urn', full_name='org.apache.beam.fn.v1.FunctionSpec.urn', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='environment_reference', full_name='org.apache.beam.fn.v1.FunctionSpec.environment_reference', index=2, - number=3, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='data', full_name='org.apache.beam.fn.v1.FunctionSpec.data', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=852, - serialized_end=958, -) - - -_SIDEINPUT = _descriptor.Descriptor( - name='SideInput', - full_name='org.apache.beam.fn.v1.SideInput', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='input', full_name='org.apache.beam.fn.v1.SideInput.input', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='view_fn', full_name='org.apache.beam.fn.v1.SideInput.view_fn', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=960, - serialized_end=1071, -) - - -_CODER = _descriptor.Descriptor( - name='Coder', - full_name='org.apache.beam.fn.v1.Coder', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='function_spec', full_name='org.apache.beam.fn.v1.Coder.function_spec', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='component_coder_reference', full_name='org.apache.beam.fn.v1.Coder.component_coder_reference', index=1, - number=2, type=9, cpp_type=9, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=1073, - serialized_end=1175, -) - - -_REMOTEGRPCPORT = _descriptor.Descriptor( - name='RemoteGrpcPort', - full_name='org.apache.beam.fn.v1.RemoteGrpcPort', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='api_service_descriptor', full_name='org.apache.beam.fn.v1.RemoteGrpcPort.api_service_descriptor', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=1177, - serialized_end=1270, -) - - -_INSTRUCTIONREQUEST = _descriptor.Descriptor( - name='InstructionRequest', - full_name='org.apache.beam.fn.v1.InstructionRequest', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='instruction_id', full_name='org.apache.beam.fn.v1.InstructionRequest.instruction_id', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='register', full_name='org.apache.beam.fn.v1.InstructionRequest.register', index=1, - number=1000, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='process_bundle', full_name='org.apache.beam.fn.v1.InstructionRequest.process_bundle', index=2, - number=1001, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='process_bundle_progress', full_name='org.apache.beam.fn.v1.InstructionRequest.process_bundle_progress', index=3, - number=1002, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='process_bundle_split', full_name='org.apache.beam.fn.v1.InstructionRequest.process_bundle_split', index=4, - number=1003, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - _descriptor.OneofDescriptor( - name='request', full_name='org.apache.beam.fn.v1.InstructionRequest.request', - index=0, containing_type=None, fields=[]), - ], - serialized_start=1273, - serialized_end=1633, -) - - -_INSTRUCTIONRESPONSE = _descriptor.Descriptor( - name='InstructionResponse', - full_name='org.apache.beam.fn.v1.InstructionResponse', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='instruction_id', full_name='org.apache.beam.fn.v1.InstructionResponse.instruction_id', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='error', full_name='org.apache.beam.fn.v1.InstructionResponse.error', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='register', full_name='org.apache.beam.fn.v1.InstructionResponse.register', index=2, - number=1000, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='process_bundle', full_name='org.apache.beam.fn.v1.InstructionResponse.process_bundle', index=3, - number=1001, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='process_bundle_progress', full_name='org.apache.beam.fn.v1.InstructionResponse.process_bundle_progress', index=4, - number=1002, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='process_bundle_split', full_name='org.apache.beam.fn.v1.InstructionResponse.process_bundle_split', index=5, - number=1003, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - _descriptor.OneofDescriptor( - name='response', full_name='org.apache.beam.fn.v1.InstructionResponse.response', - index=0, containing_type=None, fields=[]), - ], - serialized_start=1636, - serialized_end=2017, -) - - -_REGISTERREQUEST = _descriptor.Descriptor( - name='RegisterRequest', - full_name='org.apache.beam.fn.v1.RegisterRequest', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='process_bundle_descriptor', full_name='org.apache.beam.fn.v1.RegisterRequest.process_bundle_descriptor', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2019, - serialized_end=2119, -) - - -_REGISTERRESPONSE = _descriptor.Descriptor( - name='RegisterResponse', - full_name='org.apache.beam.fn.v1.RegisterResponse', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2121, - serialized_end=2139, -) - - -_PROCESSBUNDLEDESCRIPTOR = _descriptor.Descriptor( - name='ProcessBundleDescriptor', - full_name='org.apache.beam.fn.v1.ProcessBundleDescriptor', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='id', full_name='org.apache.beam.fn.v1.ProcessBundleDescriptor.id', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='primitive_transform', full_name='org.apache.beam.fn.v1.ProcessBundleDescriptor.primitive_transform', index=1, - number=2, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='coders', full_name='org.apache.beam.fn.v1.ProcessBundleDescriptor.coders', index=2, - number=4, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2142, - serialized_end=2297, -) - - -_PROCESSBUNDLEREQUEST = _descriptor.Descriptor( - name='ProcessBundleRequest', - full_name='org.apache.beam.fn.v1.ProcessBundleRequest', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='process_bundle_descriptor_reference', full_name='org.apache.beam.fn.v1.ProcessBundleRequest.process_bundle_descriptor_reference', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='cache_tokens', full_name='org.apache.beam.fn.v1.ProcessBundleRequest.cache_tokens', index=1, - number=2, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2299, - serialized_end=2423, -) - - -_PROCESSBUNDLERESPONSE = _descriptor.Descriptor( - name='ProcessBundleResponse', - full_name='org.apache.beam.fn.v1.ProcessBundleResponse', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2425, - serialized_end=2448, -) - - -_PROCESSBUNDLEPROGRESSREQUEST = _descriptor.Descriptor( - name='ProcessBundleProgressRequest', - full_name='org.apache.beam.fn.v1.ProcessBundleProgressRequest', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='instruction_reference', full_name='org.apache.beam.fn.v1.ProcessBundleProgressRequest.instruction_reference', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2450, - serialized_end=2511, -) - - -_PROCESSBUNDLEPROGRESSRESPONSE = _descriptor.Descriptor( - name='ProcessBundleProgressResponse', - full_name='org.apache.beam.fn.v1.ProcessBundleProgressResponse', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='finished_work', full_name='org.apache.beam.fn.v1.ProcessBundleProgressResponse.finished_work', index=0, - number=1, type=1, cpp_type=5, label=1, - has_default_value=False, default_value=float(0), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='backlog', full_name='org.apache.beam.fn.v1.ProcessBundleProgressResponse.backlog', index=1, - number=2, type=1, cpp_type=5, label=1, - has_default_value=False, default_value=float(0), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2513, - serialized_end=2584, -) - - -_PROCESSBUNDLESPLITREQUEST = _descriptor.Descriptor( - name='ProcessBundleSplitRequest', - full_name='org.apache.beam.fn.v1.ProcessBundleSplitRequest', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='instruction_reference', full_name='org.apache.beam.fn.v1.ProcessBundleSplitRequest.instruction_reference', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='fraction', full_name='org.apache.beam.fn.v1.ProcessBundleSplitRequest.fraction', index=1, - number=2, type=1, cpp_type=5, label=1, - has_default_value=False, default_value=float(0), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2586, - serialized_end=2662, -) - - -_ELEMENTCOUNTRESTRICTION = _descriptor.Descriptor( - name='ElementCountRestriction', - full_name='org.apache.beam.fn.v1.ElementCountRestriction', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='count', full_name='org.apache.beam.fn.v1.ElementCountRestriction.count', index=0, - number=1, type=3, cpp_type=2, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2664, - serialized_end=2704, -) - - -_ELEMENTCOUNTSKIPRESTRICTION = _descriptor.Descriptor( - name='ElementCountSkipRestriction', - full_name='org.apache.beam.fn.v1.ElementCountSkipRestriction', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='count', full_name='org.apache.beam.fn.v1.ElementCountSkipRestriction.count', index=0, - number=1, type=3, cpp_type=2, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2706, - serialized_end=2750, -) - - -_PRIMITIVETRANSFORMSPLIT = _descriptor.Descriptor( - name='PrimitiveTransformSplit', - full_name='org.apache.beam.fn.v1.PrimitiveTransformSplit', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='primitive_transform_reference', full_name='org.apache.beam.fn.v1.PrimitiveTransformSplit.primitive_transform_reference', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='completed_restriction', full_name='org.apache.beam.fn.v1.PrimitiveTransformSplit.completed_restriction', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='remaining_restriction', full_name='org.apache.beam.fn.v1.PrimitiveTransformSplit.remaining_restriction', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2753, - serialized_end=2953, -) - - -_PROCESSBUNDLESPLITRESPONSE = _descriptor.Descriptor( - name='ProcessBundleSplitResponse', - full_name='org.apache.beam.fn.v1.ProcessBundleSplitResponse', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='splits', full_name='org.apache.beam.fn.v1.ProcessBundleSplitResponse.splits', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2955, - serialized_end=3047, -) - - -_ELEMENTS_DATA = _descriptor.Descriptor( - name='Data', - full_name='org.apache.beam.fn.v1.Elements.Data', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='instruction_reference', full_name='org.apache.beam.fn.v1.Elements.Data.instruction_reference', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='target', full_name='org.apache.beam.fn.v1.Elements.Data.target', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='data', full_name='org.apache.beam.fn.v1.Elements.Data.data', index=2, - number=3, type=12, cpp_type=9, label=1, - has_default_value=False, default_value=_b(""), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3114, - serialized_end=3212, -) - -_ELEMENTS = _descriptor.Descriptor( - name='Elements', - full_name='org.apache.beam.fn.v1.Elements', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='data', full_name='org.apache.beam.fn.v1.Elements.data', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_ELEMENTS_DATA, ], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3050, - serialized_end=3212, -) - - -_STATEREQUEST = _descriptor.Descriptor( - name='StateRequest', - full_name='org.apache.beam.fn.v1.StateRequest', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='id', full_name='org.apache.beam.fn.v1.StateRequest.id', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='instruction_reference', full_name='org.apache.beam.fn.v1.StateRequest.instruction_reference', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='state_key', full_name='org.apache.beam.fn.v1.StateRequest.state_key', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='get', full_name='org.apache.beam.fn.v1.StateRequest.get', index=3, - number=1000, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='append', full_name='org.apache.beam.fn.v1.StateRequest.append', index=4, - number=1001, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='clear', full_name='org.apache.beam.fn.v1.StateRequest.clear', index=5, - number=1002, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - _descriptor.OneofDescriptor( - name='request', full_name='org.apache.beam.fn.v1.StateRequest.request', - index=0, containing_type=None, fields=[]), - ], - serialized_start=3215, - serialized_end=3513, -) - - -_STATERESPONSE = _descriptor.Descriptor( - name='StateResponse', - full_name='org.apache.beam.fn.v1.StateResponse', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='id', full_name='org.apache.beam.fn.v1.StateResponse.id', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='error', full_name='org.apache.beam.fn.v1.StateResponse.error', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='get', full_name='org.apache.beam.fn.v1.StateResponse.get', index=2, - number=1000, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='append', full_name='org.apache.beam.fn.v1.StateResponse.append', index=3, - number=1001, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='clear', full_name='org.apache.beam.fn.v1.StateResponse.clear', index=4, - number=1002, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - _descriptor.OneofDescriptor( - name='response', full_name='org.apache.beam.fn.v1.StateResponse.response', - index=0, containing_type=None, fields=[]), - ], - serialized_start=3516, - serialized_end=3751, -) - - -_CACHETOKEN = _descriptor.Descriptor( - name='CacheToken', - full_name='org.apache.beam.fn.v1.CacheToken', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='target', full_name='org.apache.beam.fn.v1.CacheToken.target', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='token', full_name='org.apache.beam.fn.v1.CacheToken.token', index=1, - number=2, type=12, cpp_type=9, label=1, - has_default_value=False, default_value=_b(""), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3753, - serialized_end=3827, -) - - -_STATEKEY = _descriptor.Descriptor( - name='StateKey', - full_name='org.apache.beam.fn.v1.StateKey', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='target', full_name='org.apache.beam.fn.v1.StateKey.target', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='window', full_name='org.apache.beam.fn.v1.StateKey.window', index=1, - number=2, type=12, cpp_type=9, label=1, - has_default_value=False, default_value=_b(""), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.fn.v1.StateKey.key', index=2, - number=3, type=12, cpp_type=9, label=1, - has_default_value=False, default_value=_b(""), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3829, - serialized_end=3915, -) - - -_CONTINUABLESTREAM = _descriptor.Descriptor( - name='ContinuableStream', - full_name='org.apache.beam.fn.v1.ContinuableStream', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='continuation_token', full_name='org.apache.beam.fn.v1.ContinuableStream.continuation_token', index=0, - number=1, type=12, cpp_type=9, label=1, - has_default_value=False, default_value=_b(""), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='data', full_name='org.apache.beam.fn.v1.ContinuableStream.data', index=1, - number=2, type=12, cpp_type=9, label=1, - has_default_value=False, default_value=_b(""), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3917, - serialized_end=3978, -) - - -_STATEGETREQUEST = _descriptor.Descriptor( - name='StateGetRequest', - full_name='org.apache.beam.fn.v1.StateGetRequest', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='continuation_token', full_name='org.apache.beam.fn.v1.StateGetRequest.continuation_token', index=0, - number=1, type=12, cpp_type=9, label=1, - has_default_value=False, default_value=_b(""), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3980, - serialized_end=4025, -) - - -_STATEGETRESPONSE = _descriptor.Descriptor( - name='StateGetResponse', - full_name='org.apache.beam.fn.v1.StateGetResponse', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='stream', full_name='org.apache.beam.fn.v1.StateGetResponse.stream', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4027, - serialized_end=4103, -) - - -_STATEAPPENDREQUEST = _descriptor.Descriptor( - name='StateAppendRequest', - full_name='org.apache.beam.fn.v1.StateAppendRequest', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='data', full_name='org.apache.beam.fn.v1.StateAppendRequest.data', index=0, - number=1, type=12, cpp_type=9, label=1, - has_default_value=False, default_value=_b(""), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4105, - serialized_end=4139, -) - - -_STATEAPPENDRESPONSE = _descriptor.Descriptor( - name='StateAppendResponse', - full_name='org.apache.beam.fn.v1.StateAppendResponse', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4141, - serialized_end=4162, -) - - -_STATECLEARREQUEST = _descriptor.Descriptor( - name='StateClearRequest', - full_name='org.apache.beam.fn.v1.StateClearRequest', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4164, - serialized_end=4183, -) - - -_STATECLEARRESPONSE = _descriptor.Descriptor( - name='StateClearResponse', - full_name='org.apache.beam.fn.v1.StateClearResponse', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4185, - serialized_end=4205, -) - - -_LOGENTRY_LIST = _descriptor.Descriptor( - name='List', - full_name='org.apache.beam.fn.v1.LogEntry.List', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='log_entries', full_name='org.apache.beam.fn.v1.LogEntry.List.log_entries', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4467, - serialized_end=4527, -) - -_LOGENTRY = _descriptor.Descriptor( - name='LogEntry', - full_name='org.apache.beam.fn.v1.LogEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='severity', full_name='org.apache.beam.fn.v1.LogEntry.severity', index=0, - number=1, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='timestamp', full_name='org.apache.beam.fn.v1.LogEntry.timestamp', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='message', full_name='org.apache.beam.fn.v1.LogEntry.message', index=2, - number=3, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='trace', full_name='org.apache.beam.fn.v1.LogEntry.trace', index=3, - number=4, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='instruction_reference', full_name='org.apache.beam.fn.v1.LogEntry.instruction_reference', index=4, - number=5, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='primitive_transform_reference', full_name='org.apache.beam.fn.v1.LogEntry.primitive_transform_reference', index=5, - number=6, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='log_location', full_name='org.apache.beam.fn.v1.LogEntry.log_location', index=6, - number=7, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='thread', full_name='org.apache.beam.fn.v1.LogEntry.thread', index=7, - number=8, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_LOGENTRY_LIST, ], - enum_types=[ - _LOGENTRY_SEVERITY, - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4208, - serialized_end=4618, -) - - -_LOGCONTROL = _descriptor.Descriptor( - name='LogControl', - full_name='org.apache.beam.fn.v1.LogControl', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4620, - serialized_end=4632, -) - - -_APISERVICEDESCRIPTOR = _descriptor.Descriptor( - name='ApiServiceDescriptor', - full_name='org.apache.beam.fn.v1.ApiServiceDescriptor', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='id', full_name='org.apache.beam.fn.v1.ApiServiceDescriptor.id', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='url', full_name='org.apache.beam.fn.v1.ApiServiceDescriptor.url', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='oauth2_client_credentials_grant', full_name='org.apache.beam.fn.v1.ApiServiceDescriptor.oauth2_client_credentials_grant', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - _descriptor.OneofDescriptor( - name='authentication', full_name='org.apache.beam.fn.v1.ApiServiceDescriptor.authentication', - index=0, containing_type=None, fields=[]), - ], - serialized_start=4635, - serialized_end=4796, -) - - -_OAUTH2CLIENTCREDENTIALSGRANT = _descriptor.Descriptor( - name='OAuth2ClientCredentialsGrant', - full_name='org.apache.beam.fn.v1.OAuth2ClientCredentialsGrant', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='url', full_name='org.apache.beam.fn.v1.OAuth2ClientCredentialsGrant.url', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4798, - serialized_end=4841, -) - - -_DOCKERCONTAINER = _descriptor.Descriptor( - name='DockerContainer', - full_name='org.apache.beam.fn.v1.DockerContainer', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='id', full_name='org.apache.beam.fn.v1.DockerContainer.id', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='uri', full_name='org.apache.beam.fn.v1.DockerContainer.uri', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='registry_reference', full_name='org.apache.beam.fn.v1.DockerContainer.registry_reference', index=2, - number=3, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4843, - serialized_end=4913, -) - -_TARGET_LIST.fields_by_name['target'].message_type = _TARGET -_TARGET_LIST.containing_type = _TARGET -_PRIMITIVETRANSFORM_INPUTSENTRY.fields_by_name['value'].message_type = _TARGET_LIST -_PRIMITIVETRANSFORM_INPUTSENTRY.containing_type = _PRIMITIVETRANSFORM -_PRIMITIVETRANSFORM_OUTPUTSENTRY.fields_by_name['value'].message_type = _PCOLLECTION -_PRIMITIVETRANSFORM_OUTPUTSENTRY.containing_type = _PRIMITIVETRANSFORM -_PRIMITIVETRANSFORM_SIDEINPUTSENTRY.fields_by_name['value'].message_type = _SIDEINPUT -_PRIMITIVETRANSFORM_SIDEINPUTSENTRY.containing_type = _PRIMITIVETRANSFORM -_PRIMITIVETRANSFORM.fields_by_name['function_spec'].message_type = _FUNCTIONSPEC -_PRIMITIVETRANSFORM.fields_by_name['inputs'].message_type = _PRIMITIVETRANSFORM_INPUTSENTRY -_PRIMITIVETRANSFORM.fields_by_name['outputs'].message_type = _PRIMITIVETRANSFORM_OUTPUTSENTRY -_PRIMITIVETRANSFORM.fields_by_name['side_inputs'].message_type = _PRIMITIVETRANSFORM_SIDEINPUTSENTRY -_FUNCTIONSPEC.fields_by_name['data'].message_type = google_dot_protobuf_dot_any__pb2._ANY -_SIDEINPUT.fields_by_name['input'].message_type = _TARGET -_SIDEINPUT.fields_by_name['view_fn'].message_type = _FUNCTIONSPEC -_CODER.fields_by_name['function_spec'].message_type = _FUNCTIONSPEC -_REMOTEGRPCPORT.fields_by_name['api_service_descriptor'].message_type = _APISERVICEDESCRIPTOR -_INSTRUCTIONREQUEST.fields_by_name['register'].message_type = _REGISTERREQUEST -_INSTRUCTIONREQUEST.fields_by_name['process_bundle'].message_type = _PROCESSBUNDLEREQUEST -_INSTRUCTIONREQUEST.fields_by_name['process_bundle_progress'].message_type = _PROCESSBUNDLEPROGRESSREQUEST -_INSTRUCTIONREQUEST.fields_by_name['process_bundle_split'].message_type = _PROCESSBUNDLESPLITREQUEST -_INSTRUCTIONREQUEST.oneofs_by_name['request'].fields.append( - _INSTRUCTIONREQUEST.fields_by_name['register']) -_INSTRUCTIONREQUEST.fields_by_name['register'].containing_oneof = _INSTRUCTIONREQUEST.oneofs_by_name['request'] -_INSTRUCTIONREQUEST.oneofs_by_name['request'].fields.append( - _INSTRUCTIONREQUEST.fields_by_name['process_bundle']) -_INSTRUCTIONREQUEST.fields_by_name['process_bundle'].containing_oneof = _INSTRUCTIONREQUEST.oneofs_by_name['request'] -_INSTRUCTIONREQUEST.oneofs_by_name['request'].fields.append( - _INSTRUCTIONREQUEST.fields_by_name['process_bundle_progress']) -_INSTRUCTIONREQUEST.fields_by_name['process_bundle_progress'].containing_oneof = _INSTRUCTIONREQUEST.oneofs_by_name['request'] -_INSTRUCTIONREQUEST.oneofs_by_name['request'].fields.append( - _INSTRUCTIONREQUEST.fields_by_name['process_bundle_split']) -_INSTRUCTIONREQUEST.fields_by_name['process_bundle_split'].containing_oneof = _INSTRUCTIONREQUEST.oneofs_by_name['request'] -_INSTRUCTIONRESPONSE.fields_by_name['register'].message_type = _REGISTERRESPONSE -_INSTRUCTIONRESPONSE.fields_by_name['process_bundle'].message_type = _PROCESSBUNDLERESPONSE -_INSTRUCTIONRESPONSE.fields_by_name['process_bundle_progress'].message_type = _PROCESSBUNDLEPROGRESSRESPONSE -_INSTRUCTIONRESPONSE.fields_by_name['process_bundle_split'].message_type = _PROCESSBUNDLESPLITRESPONSE -_INSTRUCTIONRESPONSE.oneofs_by_name['response'].fields.append( - _INSTRUCTIONRESPONSE.fields_by_name['register']) -_INSTRUCTIONRESPONSE.fields_by_name['register'].containing_oneof = _INSTRUCTIONRESPONSE.oneofs_by_name['response'] -_INSTRUCTIONRESPONSE.oneofs_by_name['response'].fields.append( - _INSTRUCTIONRESPONSE.fields_by_name['process_bundle']) -_INSTRUCTIONRESPONSE.fields_by_name['process_bundle'].containing_oneof = _INSTRUCTIONRESPONSE.oneofs_by_name['response'] -_INSTRUCTIONRESPONSE.oneofs_by_name['response'].fields.append( - _INSTRUCTIONRESPONSE.fields_by_name['process_bundle_progress']) -_INSTRUCTIONRESPONSE.fields_by_name['process_bundle_progress'].containing_oneof = _INSTRUCTIONRESPONSE.oneofs_by_name['response'] -_INSTRUCTIONRESPONSE.oneofs_by_name['response'].fields.append( - _INSTRUCTIONRESPONSE.fields_by_name['process_bundle_split']) -_INSTRUCTIONRESPONSE.fields_by_name['process_bundle_split'].containing_oneof = _INSTRUCTIONRESPONSE.oneofs_by_name['response'] -_REGISTERREQUEST.fields_by_name['process_bundle_descriptor'].message_type = _PROCESSBUNDLEDESCRIPTOR -_PROCESSBUNDLEDESCRIPTOR.fields_by_name['primitive_transform'].message_type = _PRIMITIVETRANSFORM -_PROCESSBUNDLEDESCRIPTOR.fields_by_name['coders'].message_type = _CODER -_PROCESSBUNDLEREQUEST.fields_by_name['cache_tokens'].message_type = _CACHETOKEN -_PRIMITIVETRANSFORMSPLIT.fields_by_name['completed_restriction'].message_type = _FUNCTIONSPEC -_PRIMITIVETRANSFORMSPLIT.fields_by_name['remaining_restriction'].message_type = _FUNCTIONSPEC -_PROCESSBUNDLESPLITRESPONSE.fields_by_name['splits'].message_type = _PRIMITIVETRANSFORMSPLIT -_ELEMENTS_DATA.fields_by_name['target'].message_type = _TARGET -_ELEMENTS_DATA.containing_type = _ELEMENTS -_ELEMENTS.fields_by_name['data'].message_type = _ELEMENTS_DATA -_STATEREQUEST.fields_by_name['state_key'].message_type = _STATEKEY -_STATEREQUEST.fields_by_name['get'].message_type = _STATEGETREQUEST -_STATEREQUEST.fields_by_name['append'].message_type = _STATEAPPENDREQUEST -_STATEREQUEST.fields_by_name['clear'].message_type = _STATECLEARREQUEST -_STATEREQUEST.oneofs_by_name['request'].fields.append( - _STATEREQUEST.fields_by_name['get']) -_STATEREQUEST.fields_by_name['get'].containing_oneof = _STATEREQUEST.oneofs_by_name['request'] -_STATEREQUEST.oneofs_by_name['request'].fields.append( - _STATEREQUEST.fields_by_name['append']) -_STATEREQUEST.fields_by_name['append'].containing_oneof = _STATEREQUEST.oneofs_by_name['request'] -_STATEREQUEST.oneofs_by_name['request'].fields.append( - _STATEREQUEST.fields_by_name['clear']) -_STATEREQUEST.fields_by_name['clear'].containing_oneof = _STATEREQUEST.oneofs_by_name['request'] -_STATERESPONSE.fields_by_name['get'].message_type = _STATEGETRESPONSE -_STATERESPONSE.fields_by_name['append'].message_type = _STATEAPPENDRESPONSE -_STATERESPONSE.fields_by_name['clear'].message_type = _STATECLEARRESPONSE -_STATERESPONSE.oneofs_by_name['response'].fields.append( - _STATERESPONSE.fields_by_name['get']) -_STATERESPONSE.fields_by_name['get'].containing_oneof = _STATERESPONSE.oneofs_by_name['response'] -_STATERESPONSE.oneofs_by_name['response'].fields.append( - _STATERESPONSE.fields_by_name['append']) -_STATERESPONSE.fields_by_name['append'].containing_oneof = _STATERESPONSE.oneofs_by_name['response'] -_STATERESPONSE.oneofs_by_name['response'].fields.append( - _STATERESPONSE.fields_by_name['clear']) -_STATERESPONSE.fields_by_name['clear'].containing_oneof = _STATERESPONSE.oneofs_by_name['response'] -_CACHETOKEN.fields_by_name['target'].message_type = _TARGET -_STATEKEY.fields_by_name['target'].message_type = _TARGET -_STATEGETRESPONSE.fields_by_name['stream'].message_type = _CONTINUABLESTREAM -_LOGENTRY_LIST.fields_by_name['log_entries'].message_type = _LOGENTRY -_LOGENTRY_LIST.containing_type = _LOGENTRY -_LOGENTRY.fields_by_name['severity'].enum_type = _LOGENTRY_SEVERITY -_LOGENTRY.fields_by_name['timestamp'].message_type = google_dot_protobuf_dot_timestamp__pb2._TIMESTAMP -_LOGENTRY_SEVERITY.containing_type = _LOGENTRY -_APISERVICEDESCRIPTOR.fields_by_name['oauth2_client_credentials_grant'].message_type = _OAUTH2CLIENTCREDENTIALSGRANT -_APISERVICEDESCRIPTOR.oneofs_by_name['authentication'].fields.append( - _APISERVICEDESCRIPTOR.fields_by_name['oauth2_client_credentials_grant']) -_APISERVICEDESCRIPTOR.fields_by_name['oauth2_client_credentials_grant'].containing_oneof = _APISERVICEDESCRIPTOR.oneofs_by_name['authentication'] -DESCRIPTOR.message_types_by_name['Target'] = _TARGET -DESCRIPTOR.message_types_by_name['PCollection'] = _PCOLLECTION -DESCRIPTOR.message_types_by_name['PrimitiveTransform'] = _PRIMITIVETRANSFORM -DESCRIPTOR.message_types_by_name['FunctionSpec'] = _FUNCTIONSPEC -DESCRIPTOR.message_types_by_name['SideInput'] = _SIDEINPUT -DESCRIPTOR.message_types_by_name['Coder'] = _CODER -DESCRIPTOR.message_types_by_name['RemoteGrpcPort'] = _REMOTEGRPCPORT -DESCRIPTOR.message_types_by_name['InstructionRequest'] = _INSTRUCTIONREQUEST -DESCRIPTOR.message_types_by_name['InstructionResponse'] = _INSTRUCTIONRESPONSE -DESCRIPTOR.message_types_by_name['RegisterRequest'] = _REGISTERREQUEST -DESCRIPTOR.message_types_by_name['RegisterResponse'] = _REGISTERRESPONSE -DESCRIPTOR.message_types_by_name['ProcessBundleDescriptor'] = _PROCESSBUNDLEDESCRIPTOR -DESCRIPTOR.message_types_by_name['ProcessBundleRequest'] = _PROCESSBUNDLEREQUEST -DESCRIPTOR.message_types_by_name['ProcessBundleResponse'] = _PROCESSBUNDLERESPONSE -DESCRIPTOR.message_types_by_name['ProcessBundleProgressRequest'] = _PROCESSBUNDLEPROGRESSREQUEST -DESCRIPTOR.message_types_by_name['ProcessBundleProgressResponse'] = _PROCESSBUNDLEPROGRESSRESPONSE -DESCRIPTOR.message_types_by_name['ProcessBundleSplitRequest'] = _PROCESSBUNDLESPLITREQUEST -DESCRIPTOR.message_types_by_name['ElementCountRestriction'] = _ELEMENTCOUNTRESTRICTION -DESCRIPTOR.message_types_by_name['ElementCountSkipRestriction'] = _ELEMENTCOUNTSKIPRESTRICTION -DESCRIPTOR.message_types_by_name['PrimitiveTransformSplit'] = _PRIMITIVETRANSFORMSPLIT -DESCRIPTOR.message_types_by_name['ProcessBundleSplitResponse'] = _PROCESSBUNDLESPLITRESPONSE -DESCRIPTOR.message_types_by_name['Elements'] = _ELEMENTS -DESCRIPTOR.message_types_by_name['StateRequest'] = _STATEREQUEST -DESCRIPTOR.message_types_by_name['StateResponse'] = _STATERESPONSE -DESCRIPTOR.message_types_by_name['CacheToken'] = _CACHETOKEN -DESCRIPTOR.message_types_by_name['StateKey'] = _STATEKEY -DESCRIPTOR.message_types_by_name['ContinuableStream'] = _CONTINUABLESTREAM -DESCRIPTOR.message_types_by_name['StateGetRequest'] = _STATEGETREQUEST -DESCRIPTOR.message_types_by_name['StateGetResponse'] = _STATEGETRESPONSE -DESCRIPTOR.message_types_by_name['StateAppendRequest'] = _STATEAPPENDREQUEST -DESCRIPTOR.message_types_by_name['StateAppendResponse'] = _STATEAPPENDRESPONSE -DESCRIPTOR.message_types_by_name['StateClearRequest'] = _STATECLEARREQUEST -DESCRIPTOR.message_types_by_name['StateClearResponse'] = _STATECLEARRESPONSE -DESCRIPTOR.message_types_by_name['LogEntry'] = _LOGENTRY -DESCRIPTOR.message_types_by_name['LogControl'] = _LOGCONTROL -DESCRIPTOR.message_types_by_name['ApiServiceDescriptor'] = _APISERVICEDESCRIPTOR -DESCRIPTOR.message_types_by_name['OAuth2ClientCredentialsGrant'] = _OAUTH2CLIENTCREDENTIALSGRANT -DESCRIPTOR.message_types_by_name['DockerContainer'] = _DOCKERCONTAINER - -Target = _reflection.GeneratedProtocolMessageType('Target', (_message.Message,), dict( - - List = _reflection.GeneratedProtocolMessageType('List', (_message.Message,), dict( - DESCRIPTOR = _TARGET_LIST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.Target.List) - )) - , - DESCRIPTOR = _TARGET, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.Target) - )) -_sym_db.RegisterMessage(Target) -_sym_db.RegisterMessage(Target.List) - -PCollection = _reflection.GeneratedProtocolMessageType('PCollection', (_message.Message,), dict( - DESCRIPTOR = _PCOLLECTION, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.PCollection) - )) -_sym_db.RegisterMessage(PCollection) - -PrimitiveTransform = _reflection.GeneratedProtocolMessageType('PrimitiveTransform', (_message.Message,), dict( - - InputsEntry = _reflection.GeneratedProtocolMessageType('InputsEntry', (_message.Message,), dict( - DESCRIPTOR = _PRIMITIVETRANSFORM_INPUTSENTRY, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.PrimitiveTransform.InputsEntry) - )) - , - - OutputsEntry = _reflection.GeneratedProtocolMessageType('OutputsEntry', (_message.Message,), dict( - DESCRIPTOR = _PRIMITIVETRANSFORM_OUTPUTSENTRY, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.PrimitiveTransform.OutputsEntry) - )) - , - - SideInputsEntry = _reflection.GeneratedProtocolMessageType('SideInputsEntry', (_message.Message,), dict( - DESCRIPTOR = _PRIMITIVETRANSFORM_SIDEINPUTSENTRY, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.PrimitiveTransform.SideInputsEntry) - )) - , - DESCRIPTOR = _PRIMITIVETRANSFORM, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.PrimitiveTransform) - )) -_sym_db.RegisterMessage(PrimitiveTransform) -_sym_db.RegisterMessage(PrimitiveTransform.InputsEntry) -_sym_db.RegisterMessage(PrimitiveTransform.OutputsEntry) -_sym_db.RegisterMessage(PrimitiveTransform.SideInputsEntry) - -FunctionSpec = _reflection.GeneratedProtocolMessageType('FunctionSpec', (_message.Message,), dict( - DESCRIPTOR = _FUNCTIONSPEC, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.FunctionSpec) - )) -_sym_db.RegisterMessage(FunctionSpec) - -SideInput = _reflection.GeneratedProtocolMessageType('SideInput', (_message.Message,), dict( - DESCRIPTOR = _SIDEINPUT, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.SideInput) - )) -_sym_db.RegisterMessage(SideInput) - -Coder = _reflection.GeneratedProtocolMessageType('Coder', (_message.Message,), dict( - DESCRIPTOR = _CODER, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.Coder) - )) -_sym_db.RegisterMessage(Coder) - -RemoteGrpcPort = _reflection.GeneratedProtocolMessageType('RemoteGrpcPort', (_message.Message,), dict( - DESCRIPTOR = _REMOTEGRPCPORT, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.RemoteGrpcPort) - )) -_sym_db.RegisterMessage(RemoteGrpcPort) - -InstructionRequest = _reflection.GeneratedProtocolMessageType('InstructionRequest', (_message.Message,), dict( - DESCRIPTOR = _INSTRUCTIONREQUEST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.InstructionRequest) - )) -_sym_db.RegisterMessage(InstructionRequest) - -InstructionResponse = _reflection.GeneratedProtocolMessageType('InstructionResponse', (_message.Message,), dict( - DESCRIPTOR = _INSTRUCTIONRESPONSE, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.InstructionResponse) - )) -_sym_db.RegisterMessage(InstructionResponse) - -RegisterRequest = _reflection.GeneratedProtocolMessageType('RegisterRequest', (_message.Message,), dict( - DESCRIPTOR = _REGISTERREQUEST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.RegisterRequest) - )) -_sym_db.RegisterMessage(RegisterRequest) - -RegisterResponse = _reflection.GeneratedProtocolMessageType('RegisterResponse', (_message.Message,), dict( - DESCRIPTOR = _REGISTERRESPONSE, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.RegisterResponse) - )) -_sym_db.RegisterMessage(RegisterResponse) - -ProcessBundleDescriptor = _reflection.GeneratedProtocolMessageType('ProcessBundleDescriptor', (_message.Message,), dict( - DESCRIPTOR = _PROCESSBUNDLEDESCRIPTOR, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ProcessBundleDescriptor) - )) -_sym_db.RegisterMessage(ProcessBundleDescriptor) - -ProcessBundleRequest = _reflection.GeneratedProtocolMessageType('ProcessBundleRequest', (_message.Message,), dict( - DESCRIPTOR = _PROCESSBUNDLEREQUEST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ProcessBundleRequest) - )) -_sym_db.RegisterMessage(ProcessBundleRequest) - -ProcessBundleResponse = _reflection.GeneratedProtocolMessageType('ProcessBundleResponse', (_message.Message,), dict( - DESCRIPTOR = _PROCESSBUNDLERESPONSE, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ProcessBundleResponse) - )) -_sym_db.RegisterMessage(ProcessBundleResponse) - -ProcessBundleProgressRequest = _reflection.GeneratedProtocolMessageType('ProcessBundleProgressRequest', (_message.Message,), dict( - DESCRIPTOR = _PROCESSBUNDLEPROGRESSREQUEST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ProcessBundleProgressRequest) - )) -_sym_db.RegisterMessage(ProcessBundleProgressRequest) - -ProcessBundleProgressResponse = _reflection.GeneratedProtocolMessageType('ProcessBundleProgressResponse', (_message.Message,), dict( - DESCRIPTOR = _PROCESSBUNDLEPROGRESSRESPONSE, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ProcessBundleProgressResponse) - )) -_sym_db.RegisterMessage(ProcessBundleProgressResponse) - -ProcessBundleSplitRequest = _reflection.GeneratedProtocolMessageType('ProcessBundleSplitRequest', (_message.Message,), dict( - DESCRIPTOR = _PROCESSBUNDLESPLITREQUEST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ProcessBundleSplitRequest) - )) -_sym_db.RegisterMessage(ProcessBundleSplitRequest) - -ElementCountRestriction = _reflection.GeneratedProtocolMessageType('ElementCountRestriction', (_message.Message,), dict( - DESCRIPTOR = _ELEMENTCOUNTRESTRICTION, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ElementCountRestriction) - )) -_sym_db.RegisterMessage(ElementCountRestriction) - -ElementCountSkipRestriction = _reflection.GeneratedProtocolMessageType('ElementCountSkipRestriction', (_message.Message,), dict( - DESCRIPTOR = _ELEMENTCOUNTSKIPRESTRICTION, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ElementCountSkipRestriction) - )) -_sym_db.RegisterMessage(ElementCountSkipRestriction) - -PrimitiveTransformSplit = _reflection.GeneratedProtocolMessageType('PrimitiveTransformSplit', (_message.Message,), dict( - DESCRIPTOR = _PRIMITIVETRANSFORMSPLIT, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.PrimitiveTransformSplit) - )) -_sym_db.RegisterMessage(PrimitiveTransformSplit) - -ProcessBundleSplitResponse = _reflection.GeneratedProtocolMessageType('ProcessBundleSplitResponse', (_message.Message,), dict( - DESCRIPTOR = _PROCESSBUNDLESPLITRESPONSE, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ProcessBundleSplitResponse) - )) -_sym_db.RegisterMessage(ProcessBundleSplitResponse) - -Elements = _reflection.GeneratedProtocolMessageType('Elements', (_message.Message,), dict( - - Data = _reflection.GeneratedProtocolMessageType('Data', (_message.Message,), dict( - DESCRIPTOR = _ELEMENTS_DATA, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.Elements.Data) - )) - , - DESCRIPTOR = _ELEMENTS, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.Elements) - )) -_sym_db.RegisterMessage(Elements) -_sym_db.RegisterMessage(Elements.Data) - -StateRequest = _reflection.GeneratedProtocolMessageType('StateRequest', (_message.Message,), dict( - DESCRIPTOR = _STATEREQUEST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.StateRequest) - )) -_sym_db.RegisterMessage(StateRequest) - -StateResponse = _reflection.GeneratedProtocolMessageType('StateResponse', (_message.Message,), dict( - DESCRIPTOR = _STATERESPONSE, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.StateResponse) - )) -_sym_db.RegisterMessage(StateResponse) - -CacheToken = _reflection.GeneratedProtocolMessageType('CacheToken', (_message.Message,), dict( - DESCRIPTOR = _CACHETOKEN, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.CacheToken) - )) -_sym_db.RegisterMessage(CacheToken) - -StateKey = _reflection.GeneratedProtocolMessageType('StateKey', (_message.Message,), dict( - DESCRIPTOR = _STATEKEY, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.StateKey) - )) -_sym_db.RegisterMessage(StateKey) - -ContinuableStream = _reflection.GeneratedProtocolMessageType('ContinuableStream', (_message.Message,), dict( - DESCRIPTOR = _CONTINUABLESTREAM, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ContinuableStream) - )) -_sym_db.RegisterMessage(ContinuableStream) - -StateGetRequest = _reflection.GeneratedProtocolMessageType('StateGetRequest', (_message.Message,), dict( - DESCRIPTOR = _STATEGETREQUEST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.StateGetRequest) - )) -_sym_db.RegisterMessage(StateGetRequest) - -StateGetResponse = _reflection.GeneratedProtocolMessageType('StateGetResponse', (_message.Message,), dict( - DESCRIPTOR = _STATEGETRESPONSE, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.StateGetResponse) - )) -_sym_db.RegisterMessage(StateGetResponse) - -StateAppendRequest = _reflection.GeneratedProtocolMessageType('StateAppendRequest', (_message.Message,), dict( - DESCRIPTOR = _STATEAPPENDREQUEST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.StateAppendRequest) - )) -_sym_db.RegisterMessage(StateAppendRequest) - -StateAppendResponse = _reflection.GeneratedProtocolMessageType('StateAppendResponse', (_message.Message,), dict( - DESCRIPTOR = _STATEAPPENDRESPONSE, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.StateAppendResponse) - )) -_sym_db.RegisterMessage(StateAppendResponse) - -StateClearRequest = _reflection.GeneratedProtocolMessageType('StateClearRequest', (_message.Message,), dict( - DESCRIPTOR = _STATECLEARREQUEST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.StateClearRequest) - )) -_sym_db.RegisterMessage(StateClearRequest) - -StateClearResponse = _reflection.GeneratedProtocolMessageType('StateClearResponse', (_message.Message,), dict( - DESCRIPTOR = _STATECLEARRESPONSE, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.StateClearResponse) - )) -_sym_db.RegisterMessage(StateClearResponse) - -LogEntry = _reflection.GeneratedProtocolMessageType('LogEntry', (_message.Message,), dict( - - List = _reflection.GeneratedProtocolMessageType('List', (_message.Message,), dict( - DESCRIPTOR = _LOGENTRY_LIST, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.LogEntry.List) - )) - , - DESCRIPTOR = _LOGENTRY, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.LogEntry) - )) -_sym_db.RegisterMessage(LogEntry) -_sym_db.RegisterMessage(LogEntry.List) - -LogControl = _reflection.GeneratedProtocolMessageType('LogControl', (_message.Message,), dict( - DESCRIPTOR = _LOGCONTROL, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.LogControl) - )) -_sym_db.RegisterMessage(LogControl) - -ApiServiceDescriptor = _reflection.GeneratedProtocolMessageType('ApiServiceDescriptor', (_message.Message,), dict( - DESCRIPTOR = _APISERVICEDESCRIPTOR, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.ApiServiceDescriptor) - )) -_sym_db.RegisterMessage(ApiServiceDescriptor) - -OAuth2ClientCredentialsGrant = _reflection.GeneratedProtocolMessageType('OAuth2ClientCredentialsGrant', (_message.Message,), dict( - DESCRIPTOR = _OAUTH2CLIENTCREDENTIALSGRANT, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.OAuth2ClientCredentialsGrant) - )) -_sym_db.RegisterMessage(OAuth2ClientCredentialsGrant) - -DockerContainer = _reflection.GeneratedProtocolMessageType('DockerContainer', (_message.Message,), dict( - DESCRIPTOR = _DOCKERCONTAINER, - __module__ = 'beam_fn_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.fn.v1.DockerContainer) - )) -_sym_db.RegisterMessage(DockerContainer) - - -DESCRIPTOR.has_options = True -DESCRIPTOR._options = _descriptor._ParseOptions(descriptor_pb2.FileOptions(), _b('\n\025org.apache.beam.fn.v1B\tBeamFnApi')) -_PRIMITIVETRANSFORM_INPUTSENTRY.has_options = True -_PRIMITIVETRANSFORM_INPUTSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_PRIMITIVETRANSFORM_OUTPUTSENTRY.has_options = True -_PRIMITIVETRANSFORM_OUTPUTSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_PRIMITIVETRANSFORM_SIDEINPUTSENTRY.has_options = True -_PRIMITIVETRANSFORM_SIDEINPUTSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -try: - # THESE ELEMENTS WILL BE DEPRECATED. - # Please use the generated *_pb2_grpc.py files instead. - import grpc - from grpc.framework.common import cardinality - from grpc.framework.interfaces.face import utilities as face_utilities - from grpc.beta import implementations as beta_implementations - from grpc.beta import interfaces as beta_interfaces - - - class BeamFnControlStub(object): - """ - Control Plane API - - Progress reporting and splitting still need further vetting. Also, this may change - with the addition of new types of instructions/responses related to metrics. - - An API that describes the work that a SDK harness is meant to do. - Stable - """ - - def __init__(self, channel): - """Constructor. - - Args: - channel: A grpc.Channel. - """ - self.Control = channel.stream_stream( - '/org.apache.beam.fn.v1.BeamFnControl/Control', - request_serializer=InstructionResponse.SerializeToString, - response_deserializer=InstructionRequest.FromString, - ) - - - class BeamFnControlServicer(object): - """ - Control Plane API - - Progress reporting and splitting still need further vetting. Also, this may change - with the addition of new types of instructions/responses related to metrics. - - An API that describes the work that a SDK harness is meant to do. - Stable - """ - - def Control(self, request_iterator, context): - """Instructions sent by the runner to the SDK requesting different types - of work. - """ - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - - def add_BeamFnControlServicer_to_server(servicer, server): - rpc_method_handlers = { - 'Control': grpc.stream_stream_rpc_method_handler( - servicer.Control, - request_deserializer=InstructionResponse.FromString, - response_serializer=InstructionRequest.SerializeToString, - ), - } - generic_handler = grpc.method_handlers_generic_handler( - 'org.apache.beam.fn.v1.BeamFnControl', rpc_method_handlers) - server.add_generic_rpc_handlers((generic_handler,)) - - - class BeamFnDataStub(object): - """Stable - """ - - def __init__(self, channel): - """Constructor. - - Args: - channel: A grpc.Channel. - """ - self.Data = channel.stream_stream( - '/org.apache.beam.fn.v1.BeamFnData/Data', - request_serializer=Elements.SerializeToString, - response_deserializer=Elements.FromString, - ) - - - class BeamFnDataServicer(object): - """Stable - """ - - def Data(self, request_iterator, context): - """Used to send data between harnesses. - """ - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - - def add_BeamFnDataServicer_to_server(servicer, server): - rpc_method_handlers = { - 'Data': grpc.stream_stream_rpc_method_handler( - servicer.Data, - request_deserializer=Elements.FromString, - response_serializer=Elements.SerializeToString, - ), - } - generic_handler = grpc.method_handlers_generic_handler( - 'org.apache.beam.fn.v1.BeamFnData', rpc_method_handlers) - server.add_generic_rpc_handlers((generic_handler,)) - - - class BeamFnStateStub(object): - - def __init__(self, channel): - """Constructor. - - Args: - channel: A grpc.Channel. - """ - self.State = channel.stream_stream( - '/org.apache.beam.fn.v1.BeamFnState/State', - request_serializer=StateRequest.SerializeToString, - response_deserializer=StateResponse.FromString, - ) - - - class BeamFnStateServicer(object): - - def State(self, request_iterator, context): - """Used to get/append/clear state stored by the runner on behalf of the SDK. - """ - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - - def add_BeamFnStateServicer_to_server(servicer, server): - rpc_method_handlers = { - 'State': grpc.stream_stream_rpc_method_handler( - servicer.State, - request_deserializer=StateRequest.FromString, - response_serializer=StateResponse.SerializeToString, - ), - } - generic_handler = grpc.method_handlers_generic_handler( - 'org.apache.beam.fn.v1.BeamFnState', rpc_method_handlers) - server.add_generic_rpc_handlers((generic_handler,)) - - - class BeamFnLoggingStub(object): - """Stable - """ - - def __init__(self, channel): - """Constructor. - - Args: - channel: A grpc.Channel. - """ - self.Logging = channel.stream_stream( - '/org.apache.beam.fn.v1.BeamFnLogging/Logging', - request_serializer=LogEntry.List.SerializeToString, - response_deserializer=LogControl.FromString, - ) - - - class BeamFnLoggingServicer(object): - """Stable - """ - - def Logging(self, request_iterator, context): - """Allows for the SDK to emit log entries which the runner can - associate with the active job. - """ - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - - def add_BeamFnLoggingServicer_to_server(servicer, server): - rpc_method_handlers = { - 'Logging': grpc.stream_stream_rpc_method_handler( - servicer.Logging, - request_deserializer=LogEntry.List.FromString, - response_serializer=LogControl.SerializeToString, - ), - } - generic_handler = grpc.method_handlers_generic_handler( - 'org.apache.beam.fn.v1.BeamFnLogging', rpc_method_handlers) - server.add_generic_rpc_handlers((generic_handler,)) - - - class BetaBeamFnControlServicer(object): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This class was generated - only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0.""" - """ - Control Plane API - - Progress reporting and splitting still need further vetting. Also, this may change - with the addition of new types of instructions/responses related to metrics. - - An API that describes the work that a SDK harness is meant to do. - Stable - """ - def Control(self, request_iterator, context): - """Instructions sent by the runner to the SDK requesting different types - of work. - """ - context.code(beta_interfaces.StatusCode.UNIMPLEMENTED) - - - class BetaBeamFnControlStub(object): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This class was generated - only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0.""" - """ - Control Plane API - - Progress reporting and splitting still need further vetting. Also, this may change - with the addition of new types of instructions/responses related to metrics. - - An API that describes the work that a SDK harness is meant to do. - Stable - """ - def Control(self, request_iterator, timeout, metadata=None, with_call=False, protocol_options=None): - """Instructions sent by the runner to the SDK requesting different types - of work. - """ - raise NotImplementedError() - - - def beta_create_BeamFnControl_server(servicer, pool=None, pool_size=None, default_timeout=None, maximum_timeout=None): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This function was - generated only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0""" - request_deserializers = { - ('org.apache.beam.fn.v1.BeamFnControl', 'Control'): InstructionResponse.FromString, - } - response_serializers = { - ('org.apache.beam.fn.v1.BeamFnControl', 'Control'): InstructionRequest.SerializeToString, - } - method_implementations = { - ('org.apache.beam.fn.v1.BeamFnControl', 'Control'): face_utilities.stream_stream_inline(servicer.Control), - } - server_options = beta_implementations.server_options(request_deserializers=request_deserializers, response_serializers=response_serializers, thread_pool=pool, thread_pool_size=pool_size, default_timeout=default_timeout, maximum_timeout=maximum_timeout) - return beta_implementations.server(method_implementations, options=server_options) - - - def beta_create_BeamFnControl_stub(channel, host=None, metadata_transformer=None, pool=None, pool_size=None): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This function was - generated only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0""" - request_serializers = { - ('org.apache.beam.fn.v1.BeamFnControl', 'Control'): InstructionResponse.SerializeToString, - } - response_deserializers = { - ('org.apache.beam.fn.v1.BeamFnControl', 'Control'): InstructionRequest.FromString, - } - cardinalities = { - 'Control': cardinality.Cardinality.STREAM_STREAM, - } - stub_options = beta_implementations.stub_options(host=host, metadata_transformer=metadata_transformer, request_serializers=request_serializers, response_deserializers=response_deserializers, thread_pool=pool, thread_pool_size=pool_size) - return beta_implementations.dynamic_stub(channel, 'org.apache.beam.fn.v1.BeamFnControl', cardinalities, options=stub_options) - - - class BetaBeamFnDataServicer(object): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This class was generated - only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0.""" - """Stable - """ - def Data(self, request_iterator, context): - """Used to send data between harnesses. - """ - context.code(beta_interfaces.StatusCode.UNIMPLEMENTED) - - - class BetaBeamFnDataStub(object): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This class was generated - only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0.""" - """Stable - """ - def Data(self, request_iterator, timeout, metadata=None, with_call=False, protocol_options=None): - """Used to send data between harnesses. - """ - raise NotImplementedError() - - - def beta_create_BeamFnData_server(servicer, pool=None, pool_size=None, default_timeout=None, maximum_timeout=None): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This function was - generated only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0""" - request_deserializers = { - ('org.apache.beam.fn.v1.BeamFnData', 'Data'): Elements.FromString, - } - response_serializers = { - ('org.apache.beam.fn.v1.BeamFnData', 'Data'): Elements.SerializeToString, - } - method_implementations = { - ('org.apache.beam.fn.v1.BeamFnData', 'Data'): face_utilities.stream_stream_inline(servicer.Data), - } - server_options = beta_implementations.server_options(request_deserializers=request_deserializers, response_serializers=response_serializers, thread_pool=pool, thread_pool_size=pool_size, default_timeout=default_timeout, maximum_timeout=maximum_timeout) - return beta_implementations.server(method_implementations, options=server_options) - - - def beta_create_BeamFnData_stub(channel, host=None, metadata_transformer=None, pool=None, pool_size=None): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This function was - generated only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0""" - request_serializers = { - ('org.apache.beam.fn.v1.BeamFnData', 'Data'): Elements.SerializeToString, - } - response_deserializers = { - ('org.apache.beam.fn.v1.BeamFnData', 'Data'): Elements.FromString, - } - cardinalities = { - 'Data': cardinality.Cardinality.STREAM_STREAM, - } - stub_options = beta_implementations.stub_options(host=host, metadata_transformer=metadata_transformer, request_serializers=request_serializers, response_deserializers=response_deserializers, thread_pool=pool, thread_pool_size=pool_size) - return beta_implementations.dynamic_stub(channel, 'org.apache.beam.fn.v1.BeamFnData', cardinalities, options=stub_options) - - - class BetaBeamFnStateServicer(object): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This class was generated - only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0.""" - def State(self, request_iterator, context): - """Used to get/append/clear state stored by the runner on behalf of the SDK. - """ - context.code(beta_interfaces.StatusCode.UNIMPLEMENTED) - - - class BetaBeamFnStateStub(object): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This class was generated - only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0.""" - def State(self, request_iterator, timeout, metadata=None, with_call=False, protocol_options=None): - """Used to get/append/clear state stored by the runner on behalf of the SDK. - """ - raise NotImplementedError() - - - def beta_create_BeamFnState_server(servicer, pool=None, pool_size=None, default_timeout=None, maximum_timeout=None): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This function was - generated only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0""" - request_deserializers = { - ('org.apache.beam.fn.v1.BeamFnState', 'State'): StateRequest.FromString, - } - response_serializers = { - ('org.apache.beam.fn.v1.BeamFnState', 'State'): StateResponse.SerializeToString, - } - method_implementations = { - ('org.apache.beam.fn.v1.BeamFnState', 'State'): face_utilities.stream_stream_inline(servicer.State), - } - server_options = beta_implementations.server_options(request_deserializers=request_deserializers, response_serializers=response_serializers, thread_pool=pool, thread_pool_size=pool_size, default_timeout=default_timeout, maximum_timeout=maximum_timeout) - return beta_implementations.server(method_implementations, options=server_options) - - - def beta_create_BeamFnState_stub(channel, host=None, metadata_transformer=None, pool=None, pool_size=None): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This function was - generated only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0""" - request_serializers = { - ('org.apache.beam.fn.v1.BeamFnState', 'State'): StateRequest.SerializeToString, - } - response_deserializers = { - ('org.apache.beam.fn.v1.BeamFnState', 'State'): StateResponse.FromString, - } - cardinalities = { - 'State': cardinality.Cardinality.STREAM_STREAM, - } - stub_options = beta_implementations.stub_options(host=host, metadata_transformer=metadata_transformer, request_serializers=request_serializers, response_deserializers=response_deserializers, thread_pool=pool, thread_pool_size=pool_size) - return beta_implementations.dynamic_stub(channel, 'org.apache.beam.fn.v1.BeamFnState', cardinalities, options=stub_options) - - - class BetaBeamFnLoggingServicer(object): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This class was generated - only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0.""" - """Stable - """ - def Logging(self, request_iterator, context): - """Allows for the SDK to emit log entries which the runner can - associate with the active job. - """ - context.code(beta_interfaces.StatusCode.UNIMPLEMENTED) - - - class BetaBeamFnLoggingStub(object): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This class was generated - only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0.""" - """Stable - """ - def Logging(self, request_iterator, timeout, metadata=None, with_call=False, protocol_options=None): - """Allows for the SDK to emit log entries which the runner can - associate with the active job. - """ - raise NotImplementedError() - - - def beta_create_BeamFnLogging_server(servicer, pool=None, pool_size=None, default_timeout=None, maximum_timeout=None): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This function was - generated only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0""" - request_deserializers = { - ('org.apache.beam.fn.v1.BeamFnLogging', 'Logging'): LogEntry.List.FromString, - } - response_serializers = { - ('org.apache.beam.fn.v1.BeamFnLogging', 'Logging'): LogControl.SerializeToString, - } - method_implementations = { - ('org.apache.beam.fn.v1.BeamFnLogging', 'Logging'): face_utilities.stream_stream_inline(servicer.Logging), - } - server_options = beta_implementations.server_options(request_deserializers=request_deserializers, response_serializers=response_serializers, thread_pool=pool, thread_pool_size=pool_size, default_timeout=default_timeout, maximum_timeout=maximum_timeout) - return beta_implementations.server(method_implementations, options=server_options) - - - def beta_create_BeamFnLogging_stub(channel, host=None, metadata_transformer=None, pool=None, pool_size=None): - """The Beta API is deprecated for 0.15.0 and later. - - It is recommended to use the GA API (classes and functions in this - file not marked beta) for all further purposes. This function was - generated only to ease transition from grpcio<0.15.0 to grpcio>=0.15.0""" - request_serializers = { - ('org.apache.beam.fn.v1.BeamFnLogging', 'Logging'): LogEntry.List.SerializeToString, - } - response_deserializers = { - ('org.apache.beam.fn.v1.BeamFnLogging', 'Logging'): LogControl.FromString, - } - cardinalities = { - 'Logging': cardinality.Cardinality.STREAM_STREAM, - } - stub_options = beta_implementations.stub_options(host=host, metadata_transformer=metadata_transformer, request_serializers=request_serializers, response_deserializers=response_deserializers, thread_pool=pool, thread_pool_size=pool_size) - return beta_implementations.dynamic_stub(channel, 'org.apache.beam.fn.v1.BeamFnLogging', cardinalities, options=stub_options) -except ImportError: - pass -# @@protoc_insertion_point(module_scope) diff --git a/sdks/python/apache_beam/runners/api/beam_fn_api_pb2_grpc.py b/sdks/python/apache_beam/runners/api/beam_fn_api_pb2_grpc.py deleted file mode 100644 index 08d7dada14a6..000000000000 --- a/sdks/python/apache_beam/runners/api/beam_fn_api_pb2_grpc.py +++ /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. -# - -# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT! -import grpc -from grpc.framework.common import cardinality -from grpc.framework.interfaces.face import utilities as face_utilities - -import beam_fn_api_pb2 as beam__fn__api__pb2 - -# This module is experimental. No backwards-compatibility guarantees. - - -class BeamFnControlStub(object): - """ - Control Plane API - - Progress reporting and splitting still need further vetting. Also, this may change - with the addition of new types of instructions/responses related to metrics. - - An API that describes the work that a SDK harness is meant to do. - Stable - """ - - def __init__(self, channel): - """Constructor. - - Args: - channel: A grpc.Channel. - """ - self.Control = channel.stream_stream( - '/org.apache.beam.fn.v1.BeamFnControl/Control', - request_serializer=beam__fn__api__pb2.InstructionResponse.SerializeToString, - response_deserializer=beam__fn__api__pb2.InstructionRequest.FromString, - ) - - -class BeamFnControlServicer(object): - """ - Control Plane API - - Progress reporting and splitting still need further vetting. Also, this may change - with the addition of new types of instructions/responses related to metrics. - - An API that describes the work that a SDK harness is meant to do. - Stable - """ - - def Control(self, request_iterator, context): - """Instructions sent by the runner to the SDK requesting different types - of work. - """ - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - -def add_BeamFnControlServicer_to_server(servicer, server): - rpc_method_handlers = { - 'Control': grpc.stream_stream_rpc_method_handler( - servicer.Control, - request_deserializer=beam__fn__api__pb2.InstructionResponse.FromString, - response_serializer=beam__fn__api__pb2.InstructionRequest.SerializeToString, - ), - } - generic_handler = grpc.method_handlers_generic_handler( - 'org.apache.beam.fn.v1.BeamFnControl', rpc_method_handlers) - server.add_generic_rpc_handlers((generic_handler,)) - - -class BeamFnDataStub(object): - """Stable - """ - - def __init__(self, channel): - """Constructor. - - Args: - channel: A grpc.Channel. - """ - self.Data = channel.stream_stream( - '/org.apache.beam.fn.v1.BeamFnData/Data', - request_serializer=beam__fn__api__pb2.Elements.SerializeToString, - response_deserializer=beam__fn__api__pb2.Elements.FromString, - ) - - -class BeamFnDataServicer(object): - """Stable - """ - - def Data(self, request_iterator, context): - """Used to send data between harnesses. - """ - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - -def add_BeamFnDataServicer_to_server(servicer, server): - rpc_method_handlers = { - 'Data': grpc.stream_stream_rpc_method_handler( - servicer.Data, - request_deserializer=beam__fn__api__pb2.Elements.FromString, - response_serializer=beam__fn__api__pb2.Elements.SerializeToString, - ), - } - generic_handler = grpc.method_handlers_generic_handler( - 'org.apache.beam.fn.v1.BeamFnData', rpc_method_handlers) - server.add_generic_rpc_handlers((generic_handler,)) - - -class BeamFnStateStub(object): - - def __init__(self, channel): - """Constructor. - - Args: - channel: A grpc.Channel. - """ - self.State = channel.stream_stream( - '/org.apache.beam.fn.v1.BeamFnState/State', - request_serializer=beam__fn__api__pb2.StateRequest.SerializeToString, - response_deserializer=beam__fn__api__pb2.StateResponse.FromString, - ) - - -class BeamFnStateServicer(object): - - def State(self, request_iterator, context): - """Used to get/append/clear state stored by the runner on behalf of the SDK. - """ - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - -def add_BeamFnStateServicer_to_server(servicer, server): - rpc_method_handlers = { - 'State': grpc.stream_stream_rpc_method_handler( - servicer.State, - request_deserializer=beam__fn__api__pb2.StateRequest.FromString, - response_serializer=beam__fn__api__pb2.StateResponse.SerializeToString, - ), - } - generic_handler = grpc.method_handlers_generic_handler( - 'org.apache.beam.fn.v1.BeamFnState', rpc_method_handlers) - server.add_generic_rpc_handlers((generic_handler,)) - - -class BeamFnLoggingStub(object): - """Stable - """ - - def __init__(self, channel): - """Constructor. - - Args: - channel: A grpc.Channel. - """ - self.Logging = channel.stream_stream( - '/org.apache.beam.fn.v1.BeamFnLogging/Logging', - request_serializer=beam__fn__api__pb2.LogEntry.List.SerializeToString, - response_deserializer=beam__fn__api__pb2.LogControl.FromString, - ) - - -class BeamFnLoggingServicer(object): - """Stable - """ - - def Logging(self, request_iterator, context): - """Allows for the SDK to emit log entries which the runner can - associate with the active job. - """ - context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') - - -def add_BeamFnLoggingServicer_to_server(servicer, server): - rpc_method_handlers = { - 'Logging': grpc.stream_stream_rpc_method_handler( - servicer.Logging, - request_deserializer=beam__fn__api__pb2.LogEntry.List.FromString, - response_serializer=beam__fn__api__pb2.LogControl.SerializeToString, - ), - } - generic_handler = grpc.method_handlers_generic_handler( - 'org.apache.beam.fn.v1.BeamFnLogging', rpc_method_handlers) - server.add_generic_rpc_handlers((generic_handler,)) diff --git a/sdks/python/apache_beam/runners/api/beam_runner_api_pb2.py b/sdks/python/apache_beam/runners/api/beam_runner_api_pb2.py deleted file mode 100644 index e8793b63da51..000000000000 --- a/sdks/python/apache_beam/runners/api/beam_runner_api_pb2.py +++ /dev/null @@ -1,2872 +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. -# - -# Generated by the protocol buffer compiler. DO NOT EDIT! -# source: beam_runner_api.proto - -import sys -_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) -from google.protobuf.internal import enum_type_wrapper -from google.protobuf import descriptor as _descriptor -from google.protobuf import message as _message -from google.protobuf import reflection as _reflection -from google.protobuf import symbol_database as _symbol_database -from google.protobuf import descriptor_pb2 -# @@protoc_insertion_point(imports) - -_sym_db = _symbol_database.Default() - - -from google.protobuf import any_pb2 as google_dot_protobuf_dot_any__pb2 - -# This module is experimental. No backwards-compatibility guarantees. - - -DESCRIPTOR = _descriptor.FileDescriptor( - name='beam_runner_api.proto', - package='org.apache.beam.runner_api.v1', - syntax='proto3', - serialized_pb=_b('\n\x15\x62\x65\x61m_runner_api.proto\x12\x1dorg.apache.beam.runner_api.v1\x1a\x19google/protobuf/any.proto\"\x8d\x07\n\nComponents\x12M\n\ntransforms\x18\x01 \x03(\x0b\x32\x39.org.apache.beam.runner_api.v1.Components.TransformsEntry\x12Q\n\x0cpcollections\x18\x02 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.Components.PcollectionsEntry\x12`\n\x14windowing_strategies\x18\x03 \x03(\x0b\x32\x42.org.apache.beam.runner_api.v1.Components.WindowingStrategiesEntry\x12\x45\n\x06\x63oders\x18\x04 \x03(\x0b\x32\x35.org.apache.beam.runner_api.v1.Components.CodersEntry\x12Q\n\x0c\x65nvironments\x18\x05 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.Components.EnvironmentsEntry\x1a\\\n\x0fTransformsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32).org.apache.beam.runner_api.v1.PTransform:\x02\x38\x01\x1a_\n\x11PcollectionsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x39\n\x05value\x18\x02 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.PCollection:\x02\x38\x01\x1al\n\x18WindowingStrategiesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12?\n\x05value\x18\x02 \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.WindowingStrategy:\x02\x38\x01\x1aS\n\x0b\x43odersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x33\n\x05value\x18\x02 \x01(\x0b\x32$.org.apache.beam.runner_api.v1.Coder:\x02\x38\x01\x1a_\n\x11\x45nvironmentsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x39\n\x05value\x18\x02 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.Environment:\x02\x38\x01\"\xe2\x06\n\x15MessageWithComponents\x12=\n\ncomponents\x18\x01 \x01(\x0b\x32).org.apache.beam.runner_api.v1.Components\x12\x35\n\x05\x63oder\x18\x02 \x01(\x0b\x32$.org.apache.beam.runner_api.v1.CoderH\x00\x12H\n\x0f\x63ombine_payload\x18\x03 \x01(\x0b\x32-.org.apache.beam.runner_api.v1.CombinePayloadH\x00\x12K\n\x11sdk_function_spec\x18\x04 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpecH\x00\x12\x45\n\x0epar_do_payload\x18\x06 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.ParDoPayloadH\x00\x12?\n\nptransform\x18\x07 \x01(\x0b\x32).org.apache.beam.runner_api.v1.PTransformH\x00\x12\x41\n\x0bpcollection\x18\x08 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.PCollectionH\x00\x12\x42\n\x0cread_payload\x18\t \x01(\x0b\x32*.org.apache.beam.runner_api.v1.ReadPayloadH\x00\x12>\n\nside_input\x18\x0b \x01(\x0b\x32(.org.apache.beam.runner_api.v1.SideInputH\x00\x12O\n\x13window_into_payload\x18\x0c \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.WindowIntoPayloadH\x00\x12N\n\x12windowing_strategy\x18\r \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.WindowingStrategyH\x00\x12\x44\n\rfunction_spec\x18\x0e \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpecH\x00\x42\x06\n\x04root\"\xa7\x01\n\x08Pipeline\x12=\n\ncomponents\x18\x01 \x01(\x0b\x32).org.apache.beam.runner_api.v1.Components\x12\x1a\n\x12root_transform_ids\x18\x02 \x03(\t\x12@\n\x0c\x64isplay_data\x18\x03 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.DisplayData\"\xa4\x03\n\nPTransform\x12\x13\n\x0bunique_name\x18\x05 \x01(\t\x12\x39\n\x04spec\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12\x15\n\rsubtransforms\x18\x02 \x03(\t\x12\x45\n\x06inputs\x18\x03 \x03(\x0b\x32\x35.org.apache.beam.runner_api.v1.PTransform.InputsEntry\x12G\n\x07outputs\x18\x04 \x03(\x0b\x32\x36.org.apache.beam.runner_api.v1.PTransform.OutputsEntry\x12@\n\x0c\x64isplay_data\x18\x06 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.DisplayData\x1a-\n\x0bInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a.\n\x0cOutputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xd3\x01\n\x0bPCollection\x12\x13\n\x0bunique_name\x18\x01 \x01(\t\x12\x10\n\x08\x63oder_id\x18\x02 \x01(\t\x12<\n\nis_bounded\x18\x03 \x01(\x0e\x32(.org.apache.beam.runner_api.v1.IsBounded\x12\x1d\n\x15windowing_strategy_id\x18\x04 \x01(\t\x12@\n\x0c\x64isplay_data\x18\x05 \x01(\x0b\x32*.org.apache.beam.runner_api.v1.DisplayData\"\x98\x05\n\x0cParDoPayload\x12=\n\x05\x64o_fn\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12<\n\nparameters\x18\x02 \x03(\x0b\x32(.org.apache.beam.runner_api.v1.Parameter\x12P\n\x0bside_inputs\x18\x03 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.ParDoPayload.SideInputsEntry\x12P\n\x0bstate_specs\x18\x04 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.ParDoPayload.StateSpecsEntry\x12P\n\x0btimer_specs\x18\x05 \x03(\x0b\x32;.org.apache.beam.runner_api.v1.ParDoPayload.TimerSpecsEntry\x1a[\n\x0fSideInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32(.org.apache.beam.runner_api.v1.SideInput:\x02\x38\x01\x1a[\n\x0fStateSpecsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32(.org.apache.beam.runner_api.v1.StateSpec:\x02\x38\x01\x1a[\n\x0fTimerSpecsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32(.org.apache.beam.runner_api.v1.TimerSpec:\x02\x38\x01\"\x8b\x01\n\tParameter\x12;\n\x04type\x18\x01 \x01(\x0e\x32-.org.apache.beam.runner_api.v1.Parameter.Type\"A\n\x04Type\x12\n\n\x06WINDOW\x10\x00\x12\x14\n\x10PIPELINE_OPTIONS\x10\x01\x12\x17\n\x13RESTRICTION_TRACKER\x10\x02\"\x0b\n\tStateSpec\"\x0b\n\tTimerSpec\"\x8b\x01\n\x0bReadPayload\x12>\n\x06source\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12<\n\nis_bounded\x18\x02 \x01(\x0e\x32(.org.apache.beam.runner_api.v1.IsBounded\"V\n\x11WindowIntoPayload\x12\x41\n\twindow_fn\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\"\xe1\x02\n\x0e\x43ombinePayload\x12\x42\n\ncombine_fn\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12\x1c\n\x14\x61\x63\x63umulator_coder_id\x18\x02 \x01(\t\x12<\n\nparameters\x18\x03 \x03(\x0b\x32(.org.apache.beam.runner_api.v1.Parameter\x12R\n\x0bside_inputs\x18\x04 \x03(\x0b\x32=.org.apache.beam.runner_api.v1.CombinePayload.SideInputsEntry\x1a[\n\x0fSideInputsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32(.org.apache.beam.runner_api.v1.SideInput:\x02\x38\x01\"b\n\x05\x43oder\x12<\n\x04spec\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12\x1b\n\x13\x63omponent_coder_ids\x18\x02 \x03(\t\"\xda\x03\n\x11WindowingStrategy\x12\x41\n\twindow_fn\x18\x01 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12@\n\x0cmerge_status\x18\x02 \x01(\x0e\x32*.org.apache.beam.runner_api.v1.MergeStatus\x12\x17\n\x0fwindow_coder_id\x18\x03 \x01(\t\x12\x37\n\x07trigger\x18\x04 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x12J\n\x11\x61\x63\x63umulation_mode\x18\x05 \x01(\x0e\x32/.org.apache.beam.runner_api.v1.AccumulationMode\x12>\n\x0boutput_time\x18\x06 \x01(\x0e\x32).org.apache.beam.runner_api.v1.OutputTime\x12H\n\x10\x63losing_behavior\x18\x07 \x01(\x0e\x32..org.apache.beam.runner_api.v1.ClosingBehavior\x12\x18\n\x10\x61llowed_lateness\x18\x08 \x01(\x03\"\xad\r\n\x07Trigger\x12\x44\n\tafter_all\x18\x01 \x01(\x0b\x32/.org.apache.beam.runner_api.v1.Trigger.AfterAllH\x00\x12\x44\n\tafter_any\x18\x02 \x01(\x0b\x32/.org.apache.beam.runner_api.v1.Trigger.AfterAnyH\x00\x12\x46\n\nafter_each\x18\x03 \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.Trigger.AfterEachH\x00\x12V\n\x13\x61\x66ter_end_of_window\x18\x04 \x01(\x0b\x32\x37.org.apache.beam.runner_api.v1.Trigger.AfterEndOfWindowH\x00\x12[\n\x15\x61\x66ter_processing_time\x18\x05 \x01(\x0b\x32:.org.apache.beam.runner_api.v1.Trigger.AfterProcessingTimeH\x00\x12t\n\"after_synchronized_processing_time\x18\x06 \x01(\x0b\x32\x46.org.apache.beam.runner_api.v1.Trigger.AfterSynchronizedProcessingTimeH\x00\x12?\n\x06\x61lways\x18\x0c \x01(\x0b\x32-.org.apache.beam.runner_api.v1.Trigger.AlwaysH\x00\x12\x41\n\x07\x64\x65\x66\x61ult\x18\x07 \x01(\x0b\x32..org.apache.beam.runner_api.v1.Trigger.DefaultH\x00\x12L\n\relement_count\x18\x08 \x01(\x0b\x32\x33.org.apache.beam.runner_api.v1.Trigger.ElementCountH\x00\x12=\n\x05never\x18\t \x01(\x0b\x32,.org.apache.beam.runner_api.v1.Trigger.NeverH\x00\x12\x46\n\nor_finally\x18\n \x01(\x0b\x32\x30.org.apache.beam.runner_api.v1.Trigger.OrFinallyH\x00\x12?\n\x06repeat\x18\x0b \x01(\x0b\x32-.org.apache.beam.runner_api.v1.Trigger.RepeatH\x00\x1aG\n\x08\x41\x66terAll\x12;\n\x0bsubtriggers\x18\x01 \x03(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1aG\n\x08\x41\x66terAny\x12;\n\x0bsubtriggers\x18\x01 \x03(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1aH\n\tAfterEach\x12;\n\x0bsubtriggers\x18\x01 \x03(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1a\x8f\x01\n\x10\x41\x66terEndOfWindow\x12=\n\rearly_firings\x18\x01 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x12<\n\x0clate_firings\x18\x02 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1a\x66\n\x13\x41\x66terProcessingTime\x12O\n\x14timestamp_transforms\x18\x01 \x03(\x0b\x32\x31.org.apache.beam.runner_api.v1.TimestampTransform\x1a!\n\x1f\x41\x66terSynchronizedProcessingTime\x1a\t\n\x07\x44\x65\x66\x61ult\x1a%\n\x0c\x45lementCount\x12\x15\n\relement_count\x18\x01 \x01(\x05\x1a\x07\n\x05Never\x1a\x08\n\x06\x41lways\x1az\n\tOrFinally\x12\x34\n\x04main\x18\x01 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x12\x37\n\x07\x66inally\x18\x02 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.Trigger\x1a\x44\n\x06Repeat\x12:\n\nsubtrigger\x18\x01 \x01(\x0b\x32&.org.apache.beam.runner_api.v1.TriggerB\t\n\x07trigger\"\x8e\x02\n\x12TimestampTransform\x12H\n\x05\x64\x65lay\x18\x01 \x01(\x0b\x32\x37.org.apache.beam.runner_api.v1.TimestampTransform.DelayH\x00\x12M\n\x08\x61lign_to\x18\x02 \x01(\x0b\x32\x39.org.apache.beam.runner_api.v1.TimestampTransform.AlignToH\x00\x1a\x1d\n\x05\x44\x65lay\x12\x14\n\x0c\x64\x65lay_millis\x18\x01 \x01(\x03\x1a)\n\x07\x41lignTo\x12\x0e\n\x06period\x18\x03 \x01(\x03\x12\x0e\n\x06offset\x18\x04 \x01(\x03\x42\x15\n\x13timestamp_transform\"\xdc\x01\n\tSideInput\x12\x43\n\x0e\x61\x63\x63\x65ss_pattern\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12?\n\x07view_fn\x18\x02 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\x12I\n\x11window_mapping_fn\x18\x03 \x01(\x0b\x32..org.apache.beam.runner_api.v1.SdkFunctionSpec\"\x1a\n\x0b\x45nvironment\x12\x0b\n\x03url\x18\x01 \x01(\t\"d\n\x0fSdkFunctionSpec\x12\x39\n\x04spec\x18\x01 \x01(\x0b\x32+.org.apache.beam.runner_api.v1.FunctionSpec\x12\x16\n\x0e\x65nvironment_id\x18\x02 \x01(\t\"D\n\x0c\x46unctionSpec\x12\x0b\n\x03urn\x18\x01 \x01(\t\x12\'\n\tparameter\x18\x02 \x01(\x0b\x32\x14.google.protobuf.Any\"\xf7\x03\n\x0b\x44isplayData\x12>\n\x05items\x18\x01 \x03(\x0b\x32/.org.apache.beam.runner_api.v1.DisplayData.Item\x1a\x46\n\nIdentifier\x12\x14\n\x0ctransform_id\x18\x01 \x01(\t\x12\x15\n\rtransform_urn\x18\x02 \x01(\t\x12\x0b\n\x03key\x18\x03 \x01(\t\x1a\xf9\x01\n\x04Item\x12\x41\n\x02id\x18\x01 \x01(\x0b\x32\x35.org.apache.beam.runner_api.v1.DisplayData.Identifier\x12=\n\x04type\x18\x02 \x01(\x0e\x32/.org.apache.beam.runner_api.v1.DisplayData.Type\x12#\n\x05value\x18\x03 \x01(\x0b\x32\x14.google.protobuf.Any\x12)\n\x0bshort_value\x18\x04 \x01(\x0b\x32\x14.google.protobuf.Any\x12\r\n\x05label\x18\x05 \x01(\t\x12\x10\n\x08link_url\x18\x06 \x01(\t\"d\n\x04Type\x12\n\n\x06STRING\x10\x00\x12\x0b\n\x07INTEGER\x10\x01\x12\t\n\x05\x46LOAT\x10\x02\x12\x0b\n\x07\x42OOLEAN\x10\x03\x12\r\n\tTIMESTAMP\x10\x04\x12\x0c\n\x08\x44URATION\x10\x05\x12\x0e\n\nJAVA_CLASS\x10\x06*\'\n\tIsBounded\x12\x0b\n\x07\x42OUNDED\x10\x00\x12\r\n\tUNBOUNDED\x10\x01*C\n\x0bMergeStatus\x12\x0f\n\x0bNON_MERGING\x10\x00\x12\x0f\n\x0bNEEDS_MERGE\x10\x01\x12\x12\n\x0e\x41LREADY_MERGED\x10\x02*4\n\x10\x41\x63\x63umulationMode\x12\x0e\n\nDISCARDING\x10\x00\x12\x10\n\x0c\x41\x43\x43UMULATING\x10\x01*8\n\x0f\x43losingBehavior\x12\x0f\n\x0b\x45MIT_ALWAYS\x10\x00\x12\x14\n\x10\x45MIT_IF_NONEMPTY\x10\x01*I\n\nOutputTime\x12\x11\n\rEND_OF_WINDOW\x10\x00\x12\x12\n\x0eLATEST_IN_PANE\x10\x01\x12\x14\n\x10\x45\x41RLIEST_IN_PANE\x10\x02*S\n\nTimeDomain\x12\x0e\n\nEVENT_TIME\x10\x00\x12\x13\n\x0fPROCESSING_TIME\x10\x01\x12 \n\x1cSYNCHRONIZED_PROCESSING_TIME\x10\x02\x42\x31\n$org.apache.beam.sdk.common.runner.v1B\tRunnerApib\x06proto3') - , - dependencies=[google_dot_protobuf_dot_any__pb2.DESCRIPTOR,]) -_sym_db.RegisterFileDescriptor(DESCRIPTOR) - -_ISBOUNDED = _descriptor.EnumDescriptor( - name='IsBounded', - full_name='org.apache.beam.runner_api.v1.IsBounded', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='BOUNDED', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='UNBOUNDED', index=1, number=1, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=7583, - serialized_end=7622, -) -_sym_db.RegisterEnumDescriptor(_ISBOUNDED) - -IsBounded = enum_type_wrapper.EnumTypeWrapper(_ISBOUNDED) -_MERGESTATUS = _descriptor.EnumDescriptor( - name='MergeStatus', - full_name='org.apache.beam.runner_api.v1.MergeStatus', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='NON_MERGING', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='NEEDS_MERGE', index=1, number=1, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='ALREADY_MERGED', index=2, number=2, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=7624, - serialized_end=7691, -) -_sym_db.RegisterEnumDescriptor(_MERGESTATUS) - -MergeStatus = enum_type_wrapper.EnumTypeWrapper(_MERGESTATUS) -_ACCUMULATIONMODE = _descriptor.EnumDescriptor( - name='AccumulationMode', - full_name='org.apache.beam.runner_api.v1.AccumulationMode', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='DISCARDING', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='ACCUMULATING', index=1, number=1, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=7693, - serialized_end=7745, -) -_sym_db.RegisterEnumDescriptor(_ACCUMULATIONMODE) - -AccumulationMode = enum_type_wrapper.EnumTypeWrapper(_ACCUMULATIONMODE) -_CLOSINGBEHAVIOR = _descriptor.EnumDescriptor( - name='ClosingBehavior', - full_name='org.apache.beam.runner_api.v1.ClosingBehavior', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='EMIT_ALWAYS', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='EMIT_IF_NONEMPTY', index=1, number=1, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=7747, - serialized_end=7803, -) -_sym_db.RegisterEnumDescriptor(_CLOSINGBEHAVIOR) - -ClosingBehavior = enum_type_wrapper.EnumTypeWrapper(_CLOSINGBEHAVIOR) -_OUTPUTTIME = _descriptor.EnumDescriptor( - name='OutputTime', - full_name='org.apache.beam.runner_api.v1.OutputTime', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='END_OF_WINDOW', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='LATEST_IN_PANE', index=1, number=1, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='EARLIEST_IN_PANE', index=2, number=2, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=7805, - serialized_end=7878, -) -_sym_db.RegisterEnumDescriptor(_OUTPUTTIME) - -OutputTime = enum_type_wrapper.EnumTypeWrapper(_OUTPUTTIME) -_TIMEDOMAIN = _descriptor.EnumDescriptor( - name='TimeDomain', - full_name='org.apache.beam.runner_api.v1.TimeDomain', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='EVENT_TIME', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='PROCESSING_TIME', index=1, number=1, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='SYNCHRONIZED_PROCESSING_TIME', index=2, number=2, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=7880, - serialized_end=7963, -) -_sym_db.RegisterEnumDescriptor(_TIMEDOMAIN) - -TimeDomain = enum_type_wrapper.EnumTypeWrapper(_TIMEDOMAIN) -BOUNDED = 0 -UNBOUNDED = 1 -NON_MERGING = 0 -NEEDS_MERGE = 1 -ALREADY_MERGED = 2 -DISCARDING = 0 -ACCUMULATING = 1 -EMIT_ALWAYS = 0 -EMIT_IF_NONEMPTY = 1 -END_OF_WINDOW = 0 -LATEST_IN_PANE = 1 -EARLIEST_IN_PANE = 2 -EVENT_TIME = 0 -PROCESSING_TIME = 1 -SYNCHRONIZED_PROCESSING_TIME = 2 - - -_PARAMETER_TYPE = _descriptor.EnumDescriptor( - name='Type', - full_name='org.apache.beam.runner_api.v1.Parameter.Type', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='WINDOW', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='PIPELINE_OPTIONS', index=1, number=1, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='RESTRICTION_TRACKER', index=2, number=2, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=3413, - serialized_end=3478, -) -_sym_db.RegisterEnumDescriptor(_PARAMETER_TYPE) - -_DISPLAYDATA_TYPE = _descriptor.EnumDescriptor( - name='Type', - full_name='org.apache.beam.runner_api.v1.DisplayData.Type', - filename=None, - file=DESCRIPTOR, - values=[ - _descriptor.EnumValueDescriptor( - name='STRING', index=0, number=0, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='INTEGER', index=1, number=1, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='FLOAT', index=2, number=2, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='BOOLEAN', index=3, number=3, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='TIMESTAMP', index=4, number=4, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='DURATION', index=5, number=5, - options=None, - type=None), - _descriptor.EnumValueDescriptor( - name='JAVA_CLASS', index=6, number=6, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=7481, - serialized_end=7581, -) -_sym_db.RegisterEnumDescriptor(_DISPLAYDATA_TYPE) - - -_COMPONENTS_TRANSFORMSENTRY = _descriptor.Descriptor( - name='TransformsEntry', - full_name='org.apache.beam.runner_api.v1.Components.TransformsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.Components.TransformsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.Components.TransformsEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=512, - serialized_end=604, -) - -_COMPONENTS_PCOLLECTIONSENTRY = _descriptor.Descriptor( - name='PcollectionsEntry', - full_name='org.apache.beam.runner_api.v1.Components.PcollectionsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.Components.PcollectionsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.Components.PcollectionsEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=606, - serialized_end=701, -) - -_COMPONENTS_WINDOWINGSTRATEGIESENTRY = _descriptor.Descriptor( - name='WindowingStrategiesEntry', - full_name='org.apache.beam.runner_api.v1.Components.WindowingStrategiesEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.Components.WindowingStrategiesEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.Components.WindowingStrategiesEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=703, - serialized_end=811, -) - -_COMPONENTS_CODERSENTRY = _descriptor.Descriptor( - name='CodersEntry', - full_name='org.apache.beam.runner_api.v1.Components.CodersEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.Components.CodersEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.Components.CodersEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=813, - serialized_end=896, -) - -_COMPONENTS_ENVIRONMENTSENTRY = _descriptor.Descriptor( - name='EnvironmentsEntry', - full_name='org.apache.beam.runner_api.v1.Components.EnvironmentsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.Components.EnvironmentsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.Components.EnvironmentsEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=898, - serialized_end=993, -) - -_COMPONENTS = _descriptor.Descriptor( - name='Components', - full_name='org.apache.beam.runner_api.v1.Components', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='transforms', full_name='org.apache.beam.runner_api.v1.Components.transforms', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='pcollections', full_name='org.apache.beam.runner_api.v1.Components.pcollections', index=1, - number=2, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='windowing_strategies', full_name='org.apache.beam.runner_api.v1.Components.windowing_strategies', index=2, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='coders', full_name='org.apache.beam.runner_api.v1.Components.coders', index=3, - number=4, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='environments', full_name='org.apache.beam.runner_api.v1.Components.environments', index=4, - number=5, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_COMPONENTS_TRANSFORMSENTRY, _COMPONENTS_PCOLLECTIONSENTRY, _COMPONENTS_WINDOWINGSTRATEGIESENTRY, _COMPONENTS_CODERSENTRY, _COMPONENTS_ENVIRONMENTSENTRY, ], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=84, - serialized_end=993, -) - - -_MESSAGEWITHCOMPONENTS = _descriptor.Descriptor( - name='MessageWithComponents', - full_name='org.apache.beam.runner_api.v1.MessageWithComponents', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='components', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.components', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='coder', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.coder', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='combine_payload', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.combine_payload', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='sdk_function_spec', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.sdk_function_spec', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='par_do_payload', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.par_do_payload', index=4, - number=6, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='ptransform', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.ptransform', index=5, - number=7, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='pcollection', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.pcollection', index=6, - number=8, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='read_payload', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.read_payload', index=7, - number=9, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='side_input', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.side_input', index=8, - number=11, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='window_into_payload', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.window_into_payload', index=9, - number=12, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='windowing_strategy', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.windowing_strategy', index=10, - number=13, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='function_spec', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.function_spec', index=11, - number=14, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - _descriptor.OneofDescriptor( - name='root', full_name='org.apache.beam.runner_api.v1.MessageWithComponents.root', - index=0, containing_type=None, fields=[]), - ], - serialized_start=996, - serialized_end=1862, -) - - -_PIPELINE = _descriptor.Descriptor( - name='Pipeline', - full_name='org.apache.beam.runner_api.v1.Pipeline', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='components', full_name='org.apache.beam.runner_api.v1.Pipeline.components', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='root_transform_ids', full_name='org.apache.beam.runner_api.v1.Pipeline.root_transform_ids', index=1, - number=2, type=9, cpp_type=9, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='display_data', full_name='org.apache.beam.runner_api.v1.Pipeline.display_data', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=1865, - serialized_end=2032, -) - - -_PTRANSFORM_INPUTSENTRY = _descriptor.Descriptor( - name='InputsEntry', - full_name='org.apache.beam.runner_api.v1.PTransform.InputsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.PTransform.InputsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.PTransform.InputsEntry.value', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2362, - serialized_end=2407, -) - -_PTRANSFORM_OUTPUTSENTRY = _descriptor.Descriptor( - name='OutputsEntry', - full_name='org.apache.beam.runner_api.v1.PTransform.OutputsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.PTransform.OutputsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.PTransform.OutputsEntry.value', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2409, - serialized_end=2455, -) - -_PTRANSFORM = _descriptor.Descriptor( - name='PTransform', - full_name='org.apache.beam.runner_api.v1.PTransform', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='unique_name', full_name='org.apache.beam.runner_api.v1.PTransform.unique_name', index=0, - number=5, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='spec', full_name='org.apache.beam.runner_api.v1.PTransform.spec', index=1, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='subtransforms', full_name='org.apache.beam.runner_api.v1.PTransform.subtransforms', index=2, - number=2, type=9, cpp_type=9, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='inputs', full_name='org.apache.beam.runner_api.v1.PTransform.inputs', index=3, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='outputs', full_name='org.apache.beam.runner_api.v1.PTransform.outputs', index=4, - number=4, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='display_data', full_name='org.apache.beam.runner_api.v1.PTransform.display_data', index=5, - number=6, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_PTRANSFORM_INPUTSENTRY, _PTRANSFORM_OUTPUTSENTRY, ], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2035, - serialized_end=2455, -) - - -_PCOLLECTION = _descriptor.Descriptor( - name='PCollection', - full_name='org.apache.beam.runner_api.v1.PCollection', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='unique_name', full_name='org.apache.beam.runner_api.v1.PCollection.unique_name', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='coder_id', full_name='org.apache.beam.runner_api.v1.PCollection.coder_id', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='is_bounded', full_name='org.apache.beam.runner_api.v1.PCollection.is_bounded', index=2, - number=3, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='windowing_strategy_id', full_name='org.apache.beam.runner_api.v1.PCollection.windowing_strategy_id', index=3, - number=4, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='display_data', full_name='org.apache.beam.runner_api.v1.PCollection.display_data', index=4, - number=5, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2458, - serialized_end=2669, -) - - -_PARDOPAYLOAD_SIDEINPUTSENTRY = _descriptor.Descriptor( - name='SideInputsEntry', - full_name='org.apache.beam.runner_api.v1.ParDoPayload.SideInputsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.ParDoPayload.SideInputsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.ParDoPayload.SideInputsEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3059, - serialized_end=3150, -) - -_PARDOPAYLOAD_STATESPECSENTRY = _descriptor.Descriptor( - name='StateSpecsEntry', - full_name='org.apache.beam.runner_api.v1.ParDoPayload.StateSpecsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.ParDoPayload.StateSpecsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.ParDoPayload.StateSpecsEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3152, - serialized_end=3243, -) - -_PARDOPAYLOAD_TIMERSPECSENTRY = _descriptor.Descriptor( - name='TimerSpecsEntry', - full_name='org.apache.beam.runner_api.v1.ParDoPayload.TimerSpecsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.ParDoPayload.TimerSpecsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.ParDoPayload.TimerSpecsEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3245, - serialized_end=3336, -) - -_PARDOPAYLOAD = _descriptor.Descriptor( - name='ParDoPayload', - full_name='org.apache.beam.runner_api.v1.ParDoPayload', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='do_fn', full_name='org.apache.beam.runner_api.v1.ParDoPayload.do_fn', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='parameters', full_name='org.apache.beam.runner_api.v1.ParDoPayload.parameters', index=1, - number=2, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='side_inputs', full_name='org.apache.beam.runner_api.v1.ParDoPayload.side_inputs', index=2, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='state_specs', full_name='org.apache.beam.runner_api.v1.ParDoPayload.state_specs', index=3, - number=4, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='timer_specs', full_name='org.apache.beam.runner_api.v1.ParDoPayload.timer_specs', index=4, - number=5, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_PARDOPAYLOAD_SIDEINPUTSENTRY, _PARDOPAYLOAD_STATESPECSENTRY, _PARDOPAYLOAD_TIMERSPECSENTRY, ], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=2672, - serialized_end=3336, -) - - -_PARAMETER = _descriptor.Descriptor( - name='Parameter', - full_name='org.apache.beam.runner_api.v1.Parameter', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='type', full_name='org.apache.beam.runner_api.v1.Parameter.type', index=0, - number=1, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - _PARAMETER_TYPE, - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3339, - serialized_end=3478, -) - - -_STATESPEC = _descriptor.Descriptor( - name='StateSpec', - full_name='org.apache.beam.runner_api.v1.StateSpec', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3480, - serialized_end=3491, -) - - -_TIMERSPEC = _descriptor.Descriptor( - name='TimerSpec', - full_name='org.apache.beam.runner_api.v1.TimerSpec', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3493, - serialized_end=3504, -) - - -_READPAYLOAD = _descriptor.Descriptor( - name='ReadPayload', - full_name='org.apache.beam.runner_api.v1.ReadPayload', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='source', full_name='org.apache.beam.runner_api.v1.ReadPayload.source', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='is_bounded', full_name='org.apache.beam.runner_api.v1.ReadPayload.is_bounded', index=1, - number=2, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3507, - serialized_end=3646, -) - - -_WINDOWINTOPAYLOAD = _descriptor.Descriptor( - name='WindowIntoPayload', - full_name='org.apache.beam.runner_api.v1.WindowIntoPayload', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='window_fn', full_name='org.apache.beam.runner_api.v1.WindowIntoPayload.window_fn', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3648, - serialized_end=3734, -) - - -_COMBINEPAYLOAD_SIDEINPUTSENTRY = _descriptor.Descriptor( - name='SideInputsEntry', - full_name='org.apache.beam.runner_api.v1.CombinePayload.SideInputsEntry', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.CombinePayload.SideInputsEntry.key', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.CombinePayload.SideInputsEntry.value', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3059, - serialized_end=3150, -) - -_COMBINEPAYLOAD = _descriptor.Descriptor( - name='CombinePayload', - full_name='org.apache.beam.runner_api.v1.CombinePayload', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='combine_fn', full_name='org.apache.beam.runner_api.v1.CombinePayload.combine_fn', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='accumulator_coder_id', full_name='org.apache.beam.runner_api.v1.CombinePayload.accumulator_coder_id', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='parameters', full_name='org.apache.beam.runner_api.v1.CombinePayload.parameters', index=2, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='side_inputs', full_name='org.apache.beam.runner_api.v1.CombinePayload.side_inputs', index=3, - number=4, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_COMBINEPAYLOAD_SIDEINPUTSENTRY, ], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=3737, - serialized_end=4090, -) - - -_CODER = _descriptor.Descriptor( - name='Coder', - full_name='org.apache.beam.runner_api.v1.Coder', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='spec', full_name='org.apache.beam.runner_api.v1.Coder.spec', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='component_coder_ids', full_name='org.apache.beam.runner_api.v1.Coder.component_coder_ids', index=1, - number=2, type=9, cpp_type=9, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4092, - serialized_end=4190, -) - - -_WINDOWINGSTRATEGY = _descriptor.Descriptor( - name='WindowingStrategy', - full_name='org.apache.beam.runner_api.v1.WindowingStrategy', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='window_fn', full_name='org.apache.beam.runner_api.v1.WindowingStrategy.window_fn', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='merge_status', full_name='org.apache.beam.runner_api.v1.WindowingStrategy.merge_status', index=1, - number=2, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='window_coder_id', full_name='org.apache.beam.runner_api.v1.WindowingStrategy.window_coder_id', index=2, - number=3, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='trigger', full_name='org.apache.beam.runner_api.v1.WindowingStrategy.trigger', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='accumulation_mode', full_name='org.apache.beam.runner_api.v1.WindowingStrategy.accumulation_mode', index=4, - number=5, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='output_time', full_name='org.apache.beam.runner_api.v1.WindowingStrategy.output_time', index=5, - number=6, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='closing_behavior', full_name='org.apache.beam.runner_api.v1.WindowingStrategy.closing_behavior', index=6, - number=7, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='allowed_lateness', full_name='org.apache.beam.runner_api.v1.WindowingStrategy.allowed_lateness', index=7, - number=8, type=3, cpp_type=2, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=4193, - serialized_end=4667, -) - - -_TRIGGER_AFTERALL = _descriptor.Descriptor( - name='AfterAll', - full_name='org.apache.beam.runner_api.v1.Trigger.AfterAll', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='subtriggers', full_name='org.apache.beam.runner_api.v1.Trigger.AfterAll.subtriggers', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=5602, - serialized_end=5673, -) - -_TRIGGER_AFTERANY = _descriptor.Descriptor( - name='AfterAny', - full_name='org.apache.beam.runner_api.v1.Trigger.AfterAny', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='subtriggers', full_name='org.apache.beam.runner_api.v1.Trigger.AfterAny.subtriggers', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=5675, - serialized_end=5746, -) - -_TRIGGER_AFTEREACH = _descriptor.Descriptor( - name='AfterEach', - full_name='org.apache.beam.runner_api.v1.Trigger.AfterEach', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='subtriggers', full_name='org.apache.beam.runner_api.v1.Trigger.AfterEach.subtriggers', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=5748, - serialized_end=5820, -) - -_TRIGGER_AFTERENDOFWINDOW = _descriptor.Descriptor( - name='AfterEndOfWindow', - full_name='org.apache.beam.runner_api.v1.Trigger.AfterEndOfWindow', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='early_firings', full_name='org.apache.beam.runner_api.v1.Trigger.AfterEndOfWindow.early_firings', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='late_firings', full_name='org.apache.beam.runner_api.v1.Trigger.AfterEndOfWindow.late_firings', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=5823, - serialized_end=5966, -) - -_TRIGGER_AFTERPROCESSINGTIME = _descriptor.Descriptor( - name='AfterProcessingTime', - full_name='org.apache.beam.runner_api.v1.Trigger.AfterProcessingTime', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='timestamp_transforms', full_name='org.apache.beam.runner_api.v1.Trigger.AfterProcessingTime.timestamp_transforms', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=5968, - serialized_end=6070, -) - -_TRIGGER_AFTERSYNCHRONIZEDPROCESSINGTIME = _descriptor.Descriptor( - name='AfterSynchronizedProcessingTime', - full_name='org.apache.beam.runner_api.v1.Trigger.AfterSynchronizedProcessingTime', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6072, - serialized_end=6105, -) - -_TRIGGER_DEFAULT = _descriptor.Descriptor( - name='Default', - full_name='org.apache.beam.runner_api.v1.Trigger.Default', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6107, - serialized_end=6116, -) - -_TRIGGER_ELEMENTCOUNT = _descriptor.Descriptor( - name='ElementCount', - full_name='org.apache.beam.runner_api.v1.Trigger.ElementCount', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='element_count', full_name='org.apache.beam.runner_api.v1.Trigger.ElementCount.element_count', index=0, - number=1, type=5, cpp_type=1, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6118, - serialized_end=6155, -) - -_TRIGGER_NEVER = _descriptor.Descriptor( - name='Never', - full_name='org.apache.beam.runner_api.v1.Trigger.Never', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6157, - serialized_end=6164, -) - -_TRIGGER_ALWAYS = _descriptor.Descriptor( - name='Always', - full_name='org.apache.beam.runner_api.v1.Trigger.Always', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6166, - serialized_end=6174, -) - -_TRIGGER_ORFINALLY = _descriptor.Descriptor( - name='OrFinally', - full_name='org.apache.beam.runner_api.v1.Trigger.OrFinally', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='main', full_name='org.apache.beam.runner_api.v1.Trigger.OrFinally.main', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='finally', full_name='org.apache.beam.runner_api.v1.Trigger.OrFinally.finally', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6176, - serialized_end=6298, -) - -_TRIGGER_REPEAT = _descriptor.Descriptor( - name='Repeat', - full_name='org.apache.beam.runner_api.v1.Trigger.Repeat', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='subtrigger', full_name='org.apache.beam.runner_api.v1.Trigger.Repeat.subtrigger', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6300, - serialized_end=6368, -) - -_TRIGGER = _descriptor.Descriptor( - name='Trigger', - full_name='org.apache.beam.runner_api.v1.Trigger', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='after_all', full_name='org.apache.beam.runner_api.v1.Trigger.after_all', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='after_any', full_name='org.apache.beam.runner_api.v1.Trigger.after_any', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='after_each', full_name='org.apache.beam.runner_api.v1.Trigger.after_each', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='after_end_of_window', full_name='org.apache.beam.runner_api.v1.Trigger.after_end_of_window', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='after_processing_time', full_name='org.apache.beam.runner_api.v1.Trigger.after_processing_time', index=4, - number=5, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='after_synchronized_processing_time', full_name='org.apache.beam.runner_api.v1.Trigger.after_synchronized_processing_time', index=5, - number=6, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='always', full_name='org.apache.beam.runner_api.v1.Trigger.always', index=6, - number=12, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='default', full_name='org.apache.beam.runner_api.v1.Trigger.default', index=7, - number=7, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='element_count', full_name='org.apache.beam.runner_api.v1.Trigger.element_count', index=8, - number=8, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='never', full_name='org.apache.beam.runner_api.v1.Trigger.never', index=9, - number=9, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='or_finally', full_name='org.apache.beam.runner_api.v1.Trigger.or_finally', index=10, - number=10, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='repeat', full_name='org.apache.beam.runner_api.v1.Trigger.repeat', index=11, - number=11, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_TRIGGER_AFTERALL, _TRIGGER_AFTERANY, _TRIGGER_AFTEREACH, _TRIGGER_AFTERENDOFWINDOW, _TRIGGER_AFTERPROCESSINGTIME, _TRIGGER_AFTERSYNCHRONIZEDPROCESSINGTIME, _TRIGGER_DEFAULT, _TRIGGER_ELEMENTCOUNT, _TRIGGER_NEVER, _TRIGGER_ALWAYS, _TRIGGER_ORFINALLY, _TRIGGER_REPEAT, ], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - _descriptor.OneofDescriptor( - name='trigger', full_name='org.apache.beam.runner_api.v1.Trigger.trigger', - index=0, containing_type=None, fields=[]), - ], - serialized_start=4670, - serialized_end=6379, -) - - -_TIMESTAMPTRANSFORM_DELAY = _descriptor.Descriptor( - name='Delay', - full_name='org.apache.beam.runner_api.v1.TimestampTransform.Delay', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='delay_millis', full_name='org.apache.beam.runner_api.v1.TimestampTransform.Delay.delay_millis', index=0, - number=1, type=3, cpp_type=2, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6557, - serialized_end=6586, -) - -_TIMESTAMPTRANSFORM_ALIGNTO = _descriptor.Descriptor( - name='AlignTo', - full_name='org.apache.beam.runner_api.v1.TimestampTransform.AlignTo', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='period', full_name='org.apache.beam.runner_api.v1.TimestampTransform.AlignTo.period', index=0, - number=3, type=3, cpp_type=2, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='offset', full_name='org.apache.beam.runner_api.v1.TimestampTransform.AlignTo.offset', index=1, - number=4, type=3, cpp_type=2, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6588, - serialized_end=6629, -) - -_TIMESTAMPTRANSFORM = _descriptor.Descriptor( - name='TimestampTransform', - full_name='org.apache.beam.runner_api.v1.TimestampTransform', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='delay', full_name='org.apache.beam.runner_api.v1.TimestampTransform.delay', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='align_to', full_name='org.apache.beam.runner_api.v1.TimestampTransform.align_to', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_TIMESTAMPTRANSFORM_DELAY, _TIMESTAMPTRANSFORM_ALIGNTO, ], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - _descriptor.OneofDescriptor( - name='timestamp_transform', full_name='org.apache.beam.runner_api.v1.TimestampTransform.timestamp_transform', - index=0, containing_type=None, fields=[]), - ], - serialized_start=6382, - serialized_end=6652, -) - - -_SIDEINPUT = _descriptor.Descriptor( - name='SideInput', - full_name='org.apache.beam.runner_api.v1.SideInput', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='access_pattern', full_name='org.apache.beam.runner_api.v1.SideInput.access_pattern', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='view_fn', full_name='org.apache.beam.runner_api.v1.SideInput.view_fn', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='window_mapping_fn', full_name='org.apache.beam.runner_api.v1.SideInput.window_mapping_fn', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6655, - serialized_end=6875, -) - - -_ENVIRONMENT = _descriptor.Descriptor( - name='Environment', - full_name='org.apache.beam.runner_api.v1.Environment', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='url', full_name='org.apache.beam.runner_api.v1.Environment.url', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6877, - serialized_end=6903, -) - - -_SDKFUNCTIONSPEC = _descriptor.Descriptor( - name='SdkFunctionSpec', - full_name='org.apache.beam.runner_api.v1.SdkFunctionSpec', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='spec', full_name='org.apache.beam.runner_api.v1.SdkFunctionSpec.spec', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='environment_id', full_name='org.apache.beam.runner_api.v1.SdkFunctionSpec.environment_id', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=6905, - serialized_end=7005, -) - - -_FUNCTIONSPEC = _descriptor.Descriptor( - name='FunctionSpec', - full_name='org.apache.beam.runner_api.v1.FunctionSpec', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='urn', full_name='org.apache.beam.runner_api.v1.FunctionSpec.urn', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='parameter', full_name='org.apache.beam.runner_api.v1.FunctionSpec.parameter', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=7007, - serialized_end=7075, -) - - -_DISPLAYDATA_IDENTIFIER = _descriptor.Descriptor( - name='Identifier', - full_name='org.apache.beam.runner_api.v1.DisplayData.Identifier', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='transform_id', full_name='org.apache.beam.runner_api.v1.DisplayData.Identifier.transform_id', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='transform_urn', full_name='org.apache.beam.runner_api.v1.DisplayData.Identifier.transform_urn', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='key', full_name='org.apache.beam.runner_api.v1.DisplayData.Identifier.key', index=2, - number=3, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=7157, - serialized_end=7227, -) - -_DISPLAYDATA_ITEM = _descriptor.Descriptor( - name='Item', - full_name='org.apache.beam.runner_api.v1.DisplayData.Item', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='id', full_name='org.apache.beam.runner_api.v1.DisplayData.Item.id', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='type', full_name='org.apache.beam.runner_api.v1.DisplayData.Item.type', index=1, - number=2, type=14, cpp_type=8, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='org.apache.beam.runner_api.v1.DisplayData.Item.value', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='short_value', full_name='org.apache.beam.runner_api.v1.DisplayData.Item.short_value', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='label', full_name='org.apache.beam.runner_api.v1.DisplayData.Item.label', index=4, - number=5, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='link_url', full_name='org.apache.beam.runner_api.v1.DisplayData.Item.link_url', index=5, - number=6, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=7230, - serialized_end=7479, -) - -_DISPLAYDATA = _descriptor.Descriptor( - name='DisplayData', - full_name='org.apache.beam.runner_api.v1.DisplayData', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='items', full_name='org.apache.beam.runner_api.v1.DisplayData.items', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_DISPLAYDATA_IDENTIFIER, _DISPLAYDATA_ITEM, ], - enum_types=[ - _DISPLAYDATA_TYPE, - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=7078, - serialized_end=7581, -) - -_COMPONENTS_TRANSFORMSENTRY.fields_by_name['value'].message_type = _PTRANSFORM -_COMPONENTS_TRANSFORMSENTRY.containing_type = _COMPONENTS -_COMPONENTS_PCOLLECTIONSENTRY.fields_by_name['value'].message_type = _PCOLLECTION -_COMPONENTS_PCOLLECTIONSENTRY.containing_type = _COMPONENTS -_COMPONENTS_WINDOWINGSTRATEGIESENTRY.fields_by_name['value'].message_type = _WINDOWINGSTRATEGY -_COMPONENTS_WINDOWINGSTRATEGIESENTRY.containing_type = _COMPONENTS -_COMPONENTS_CODERSENTRY.fields_by_name['value'].message_type = _CODER -_COMPONENTS_CODERSENTRY.containing_type = _COMPONENTS -_COMPONENTS_ENVIRONMENTSENTRY.fields_by_name['value'].message_type = _ENVIRONMENT -_COMPONENTS_ENVIRONMENTSENTRY.containing_type = _COMPONENTS -_COMPONENTS.fields_by_name['transforms'].message_type = _COMPONENTS_TRANSFORMSENTRY -_COMPONENTS.fields_by_name['pcollections'].message_type = _COMPONENTS_PCOLLECTIONSENTRY -_COMPONENTS.fields_by_name['windowing_strategies'].message_type = _COMPONENTS_WINDOWINGSTRATEGIESENTRY -_COMPONENTS.fields_by_name['coders'].message_type = _COMPONENTS_CODERSENTRY -_COMPONENTS.fields_by_name['environments'].message_type = _COMPONENTS_ENVIRONMENTSENTRY -_MESSAGEWITHCOMPONENTS.fields_by_name['components'].message_type = _COMPONENTS -_MESSAGEWITHCOMPONENTS.fields_by_name['coder'].message_type = _CODER -_MESSAGEWITHCOMPONENTS.fields_by_name['combine_payload'].message_type = _COMBINEPAYLOAD -_MESSAGEWITHCOMPONENTS.fields_by_name['sdk_function_spec'].message_type = _SDKFUNCTIONSPEC -_MESSAGEWITHCOMPONENTS.fields_by_name['par_do_payload'].message_type = _PARDOPAYLOAD -_MESSAGEWITHCOMPONENTS.fields_by_name['ptransform'].message_type = _PTRANSFORM -_MESSAGEWITHCOMPONENTS.fields_by_name['pcollection'].message_type = _PCOLLECTION -_MESSAGEWITHCOMPONENTS.fields_by_name['read_payload'].message_type = _READPAYLOAD -_MESSAGEWITHCOMPONENTS.fields_by_name['side_input'].message_type = _SIDEINPUT -_MESSAGEWITHCOMPONENTS.fields_by_name['window_into_payload'].message_type = _WINDOWINTOPAYLOAD -_MESSAGEWITHCOMPONENTS.fields_by_name['windowing_strategy'].message_type = _WINDOWINGSTRATEGY -_MESSAGEWITHCOMPONENTS.fields_by_name['function_spec'].message_type = _FUNCTIONSPEC -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['coder']) -_MESSAGEWITHCOMPONENTS.fields_by_name['coder'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['combine_payload']) -_MESSAGEWITHCOMPONENTS.fields_by_name['combine_payload'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['sdk_function_spec']) -_MESSAGEWITHCOMPONENTS.fields_by_name['sdk_function_spec'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['par_do_payload']) -_MESSAGEWITHCOMPONENTS.fields_by_name['par_do_payload'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['ptransform']) -_MESSAGEWITHCOMPONENTS.fields_by_name['ptransform'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['pcollection']) -_MESSAGEWITHCOMPONENTS.fields_by_name['pcollection'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['read_payload']) -_MESSAGEWITHCOMPONENTS.fields_by_name['read_payload'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['side_input']) -_MESSAGEWITHCOMPONENTS.fields_by_name['side_input'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['window_into_payload']) -_MESSAGEWITHCOMPONENTS.fields_by_name['window_into_payload'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['windowing_strategy']) -_MESSAGEWITHCOMPONENTS.fields_by_name['windowing_strategy'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_MESSAGEWITHCOMPONENTS.oneofs_by_name['root'].fields.append( - _MESSAGEWITHCOMPONENTS.fields_by_name['function_spec']) -_MESSAGEWITHCOMPONENTS.fields_by_name['function_spec'].containing_oneof = _MESSAGEWITHCOMPONENTS.oneofs_by_name['root'] -_PIPELINE.fields_by_name['components'].message_type = _COMPONENTS -_PIPELINE.fields_by_name['display_data'].message_type = _DISPLAYDATA -_PTRANSFORM_INPUTSENTRY.containing_type = _PTRANSFORM -_PTRANSFORM_OUTPUTSENTRY.containing_type = _PTRANSFORM -_PTRANSFORM.fields_by_name['spec'].message_type = _FUNCTIONSPEC -_PTRANSFORM.fields_by_name['inputs'].message_type = _PTRANSFORM_INPUTSENTRY -_PTRANSFORM.fields_by_name['outputs'].message_type = _PTRANSFORM_OUTPUTSENTRY -_PTRANSFORM.fields_by_name['display_data'].message_type = _DISPLAYDATA -_PCOLLECTION.fields_by_name['is_bounded'].enum_type = _ISBOUNDED -_PCOLLECTION.fields_by_name['display_data'].message_type = _DISPLAYDATA -_PARDOPAYLOAD_SIDEINPUTSENTRY.fields_by_name['value'].message_type = _SIDEINPUT -_PARDOPAYLOAD_SIDEINPUTSENTRY.containing_type = _PARDOPAYLOAD -_PARDOPAYLOAD_STATESPECSENTRY.fields_by_name['value'].message_type = _STATESPEC -_PARDOPAYLOAD_STATESPECSENTRY.containing_type = _PARDOPAYLOAD -_PARDOPAYLOAD_TIMERSPECSENTRY.fields_by_name['value'].message_type = _TIMERSPEC -_PARDOPAYLOAD_TIMERSPECSENTRY.containing_type = _PARDOPAYLOAD -_PARDOPAYLOAD.fields_by_name['do_fn'].message_type = _SDKFUNCTIONSPEC -_PARDOPAYLOAD.fields_by_name['parameters'].message_type = _PARAMETER -_PARDOPAYLOAD.fields_by_name['side_inputs'].message_type = _PARDOPAYLOAD_SIDEINPUTSENTRY -_PARDOPAYLOAD.fields_by_name['state_specs'].message_type = _PARDOPAYLOAD_STATESPECSENTRY -_PARDOPAYLOAD.fields_by_name['timer_specs'].message_type = _PARDOPAYLOAD_TIMERSPECSENTRY -_PARAMETER.fields_by_name['type'].enum_type = _PARAMETER_TYPE -_PARAMETER_TYPE.containing_type = _PARAMETER -_READPAYLOAD.fields_by_name['source'].message_type = _SDKFUNCTIONSPEC -_READPAYLOAD.fields_by_name['is_bounded'].enum_type = _ISBOUNDED -_WINDOWINTOPAYLOAD.fields_by_name['window_fn'].message_type = _SDKFUNCTIONSPEC -_COMBINEPAYLOAD_SIDEINPUTSENTRY.fields_by_name['value'].message_type = _SIDEINPUT -_COMBINEPAYLOAD_SIDEINPUTSENTRY.containing_type = _COMBINEPAYLOAD -_COMBINEPAYLOAD.fields_by_name['combine_fn'].message_type = _SDKFUNCTIONSPEC -_COMBINEPAYLOAD.fields_by_name['parameters'].message_type = _PARAMETER -_COMBINEPAYLOAD.fields_by_name['side_inputs'].message_type = _COMBINEPAYLOAD_SIDEINPUTSENTRY -_CODER.fields_by_name['spec'].message_type = _SDKFUNCTIONSPEC -_WINDOWINGSTRATEGY.fields_by_name['window_fn'].message_type = _SDKFUNCTIONSPEC -_WINDOWINGSTRATEGY.fields_by_name['merge_status'].enum_type = _MERGESTATUS -_WINDOWINGSTRATEGY.fields_by_name['trigger'].message_type = _TRIGGER -_WINDOWINGSTRATEGY.fields_by_name['accumulation_mode'].enum_type = _ACCUMULATIONMODE -_WINDOWINGSTRATEGY.fields_by_name['output_time'].enum_type = _OUTPUTTIME -_WINDOWINGSTRATEGY.fields_by_name['closing_behavior'].enum_type = _CLOSINGBEHAVIOR -_TRIGGER_AFTERALL.fields_by_name['subtriggers'].message_type = _TRIGGER -_TRIGGER_AFTERALL.containing_type = _TRIGGER -_TRIGGER_AFTERANY.fields_by_name['subtriggers'].message_type = _TRIGGER -_TRIGGER_AFTERANY.containing_type = _TRIGGER -_TRIGGER_AFTEREACH.fields_by_name['subtriggers'].message_type = _TRIGGER -_TRIGGER_AFTEREACH.containing_type = _TRIGGER -_TRIGGER_AFTERENDOFWINDOW.fields_by_name['early_firings'].message_type = _TRIGGER -_TRIGGER_AFTERENDOFWINDOW.fields_by_name['late_firings'].message_type = _TRIGGER -_TRIGGER_AFTERENDOFWINDOW.containing_type = _TRIGGER -_TRIGGER_AFTERPROCESSINGTIME.fields_by_name['timestamp_transforms'].message_type = _TIMESTAMPTRANSFORM -_TRIGGER_AFTERPROCESSINGTIME.containing_type = _TRIGGER -_TRIGGER_AFTERSYNCHRONIZEDPROCESSINGTIME.containing_type = _TRIGGER -_TRIGGER_DEFAULT.containing_type = _TRIGGER -_TRIGGER_ELEMENTCOUNT.containing_type = _TRIGGER -_TRIGGER_NEVER.containing_type = _TRIGGER -_TRIGGER_ALWAYS.containing_type = _TRIGGER -_TRIGGER_ORFINALLY.fields_by_name['main'].message_type = _TRIGGER -_TRIGGER_ORFINALLY.fields_by_name['finally'].message_type = _TRIGGER -_TRIGGER_ORFINALLY.containing_type = _TRIGGER -_TRIGGER_REPEAT.fields_by_name['subtrigger'].message_type = _TRIGGER -_TRIGGER_REPEAT.containing_type = _TRIGGER -_TRIGGER.fields_by_name['after_all'].message_type = _TRIGGER_AFTERALL -_TRIGGER.fields_by_name['after_any'].message_type = _TRIGGER_AFTERANY -_TRIGGER.fields_by_name['after_each'].message_type = _TRIGGER_AFTEREACH -_TRIGGER.fields_by_name['after_end_of_window'].message_type = _TRIGGER_AFTERENDOFWINDOW -_TRIGGER.fields_by_name['after_processing_time'].message_type = _TRIGGER_AFTERPROCESSINGTIME -_TRIGGER.fields_by_name['after_synchronized_processing_time'].message_type = _TRIGGER_AFTERSYNCHRONIZEDPROCESSINGTIME -_TRIGGER.fields_by_name['always'].message_type = _TRIGGER_ALWAYS -_TRIGGER.fields_by_name['default'].message_type = _TRIGGER_DEFAULT -_TRIGGER.fields_by_name['element_count'].message_type = _TRIGGER_ELEMENTCOUNT -_TRIGGER.fields_by_name['never'].message_type = _TRIGGER_NEVER -_TRIGGER.fields_by_name['or_finally'].message_type = _TRIGGER_ORFINALLY -_TRIGGER.fields_by_name['repeat'].message_type = _TRIGGER_REPEAT -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['after_all']) -_TRIGGER.fields_by_name['after_all'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['after_any']) -_TRIGGER.fields_by_name['after_any'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['after_each']) -_TRIGGER.fields_by_name['after_each'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['after_end_of_window']) -_TRIGGER.fields_by_name['after_end_of_window'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['after_processing_time']) -_TRIGGER.fields_by_name['after_processing_time'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['after_synchronized_processing_time']) -_TRIGGER.fields_by_name['after_synchronized_processing_time'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['always']) -_TRIGGER.fields_by_name['always'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['default']) -_TRIGGER.fields_by_name['default'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['element_count']) -_TRIGGER.fields_by_name['element_count'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['never']) -_TRIGGER.fields_by_name['never'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['or_finally']) -_TRIGGER.fields_by_name['or_finally'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TRIGGER.oneofs_by_name['trigger'].fields.append( - _TRIGGER.fields_by_name['repeat']) -_TRIGGER.fields_by_name['repeat'].containing_oneof = _TRIGGER.oneofs_by_name['trigger'] -_TIMESTAMPTRANSFORM_DELAY.containing_type = _TIMESTAMPTRANSFORM -_TIMESTAMPTRANSFORM_ALIGNTO.containing_type = _TIMESTAMPTRANSFORM -_TIMESTAMPTRANSFORM.fields_by_name['delay'].message_type = _TIMESTAMPTRANSFORM_DELAY -_TIMESTAMPTRANSFORM.fields_by_name['align_to'].message_type = _TIMESTAMPTRANSFORM_ALIGNTO -_TIMESTAMPTRANSFORM.oneofs_by_name['timestamp_transform'].fields.append( - _TIMESTAMPTRANSFORM.fields_by_name['delay']) -_TIMESTAMPTRANSFORM.fields_by_name['delay'].containing_oneof = _TIMESTAMPTRANSFORM.oneofs_by_name['timestamp_transform'] -_TIMESTAMPTRANSFORM.oneofs_by_name['timestamp_transform'].fields.append( - _TIMESTAMPTRANSFORM.fields_by_name['align_to']) -_TIMESTAMPTRANSFORM.fields_by_name['align_to'].containing_oneof = _TIMESTAMPTRANSFORM.oneofs_by_name['timestamp_transform'] -_SIDEINPUT.fields_by_name['access_pattern'].message_type = _FUNCTIONSPEC -_SIDEINPUT.fields_by_name['view_fn'].message_type = _SDKFUNCTIONSPEC -_SIDEINPUT.fields_by_name['window_mapping_fn'].message_type = _SDKFUNCTIONSPEC -_SDKFUNCTIONSPEC.fields_by_name['spec'].message_type = _FUNCTIONSPEC -_FUNCTIONSPEC.fields_by_name['parameter'].message_type = google_dot_protobuf_dot_any__pb2._ANY -_DISPLAYDATA_IDENTIFIER.containing_type = _DISPLAYDATA -_DISPLAYDATA_ITEM.fields_by_name['id'].message_type = _DISPLAYDATA_IDENTIFIER -_DISPLAYDATA_ITEM.fields_by_name['type'].enum_type = _DISPLAYDATA_TYPE -_DISPLAYDATA_ITEM.fields_by_name['value'].message_type = google_dot_protobuf_dot_any__pb2._ANY -_DISPLAYDATA_ITEM.fields_by_name['short_value'].message_type = google_dot_protobuf_dot_any__pb2._ANY -_DISPLAYDATA_ITEM.containing_type = _DISPLAYDATA -_DISPLAYDATA.fields_by_name['items'].message_type = _DISPLAYDATA_ITEM -_DISPLAYDATA_TYPE.containing_type = _DISPLAYDATA -DESCRIPTOR.message_types_by_name['Components'] = _COMPONENTS -DESCRIPTOR.message_types_by_name['MessageWithComponents'] = _MESSAGEWITHCOMPONENTS -DESCRIPTOR.message_types_by_name['Pipeline'] = _PIPELINE -DESCRIPTOR.message_types_by_name['PTransform'] = _PTRANSFORM -DESCRIPTOR.message_types_by_name['PCollection'] = _PCOLLECTION -DESCRIPTOR.message_types_by_name['ParDoPayload'] = _PARDOPAYLOAD -DESCRIPTOR.message_types_by_name['Parameter'] = _PARAMETER -DESCRIPTOR.message_types_by_name['StateSpec'] = _STATESPEC -DESCRIPTOR.message_types_by_name['TimerSpec'] = _TIMERSPEC -DESCRIPTOR.message_types_by_name['ReadPayload'] = _READPAYLOAD -DESCRIPTOR.message_types_by_name['WindowIntoPayload'] = _WINDOWINTOPAYLOAD -DESCRIPTOR.message_types_by_name['CombinePayload'] = _COMBINEPAYLOAD -DESCRIPTOR.message_types_by_name['Coder'] = _CODER -DESCRIPTOR.message_types_by_name['WindowingStrategy'] = _WINDOWINGSTRATEGY -DESCRIPTOR.message_types_by_name['Trigger'] = _TRIGGER -DESCRIPTOR.message_types_by_name['TimestampTransform'] = _TIMESTAMPTRANSFORM -DESCRIPTOR.message_types_by_name['SideInput'] = _SIDEINPUT -DESCRIPTOR.message_types_by_name['Environment'] = _ENVIRONMENT -DESCRIPTOR.message_types_by_name['SdkFunctionSpec'] = _SDKFUNCTIONSPEC -DESCRIPTOR.message_types_by_name['FunctionSpec'] = _FUNCTIONSPEC -DESCRIPTOR.message_types_by_name['DisplayData'] = _DISPLAYDATA -DESCRIPTOR.enum_types_by_name['IsBounded'] = _ISBOUNDED -DESCRIPTOR.enum_types_by_name['MergeStatus'] = _MERGESTATUS -DESCRIPTOR.enum_types_by_name['AccumulationMode'] = _ACCUMULATIONMODE -DESCRIPTOR.enum_types_by_name['ClosingBehavior'] = _CLOSINGBEHAVIOR -DESCRIPTOR.enum_types_by_name['OutputTime'] = _OUTPUTTIME -DESCRIPTOR.enum_types_by_name['TimeDomain'] = _TIMEDOMAIN - -Components = _reflection.GeneratedProtocolMessageType('Components', (_message.Message,), dict( - - TransformsEntry = _reflection.GeneratedProtocolMessageType('TransformsEntry', (_message.Message,), dict( - DESCRIPTOR = _COMPONENTS_TRANSFORMSENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Components.TransformsEntry) - )) - , - - PcollectionsEntry = _reflection.GeneratedProtocolMessageType('PcollectionsEntry', (_message.Message,), dict( - DESCRIPTOR = _COMPONENTS_PCOLLECTIONSENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Components.PcollectionsEntry) - )) - , - - WindowingStrategiesEntry = _reflection.GeneratedProtocolMessageType('WindowingStrategiesEntry', (_message.Message,), dict( - DESCRIPTOR = _COMPONENTS_WINDOWINGSTRATEGIESENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Components.WindowingStrategiesEntry) - )) - , - - CodersEntry = _reflection.GeneratedProtocolMessageType('CodersEntry', (_message.Message,), dict( - DESCRIPTOR = _COMPONENTS_CODERSENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Components.CodersEntry) - )) - , - - EnvironmentsEntry = _reflection.GeneratedProtocolMessageType('EnvironmentsEntry', (_message.Message,), dict( - DESCRIPTOR = _COMPONENTS_ENVIRONMENTSENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Components.EnvironmentsEntry) - )) - , - DESCRIPTOR = _COMPONENTS, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Components) - )) -_sym_db.RegisterMessage(Components) -_sym_db.RegisterMessage(Components.TransformsEntry) -_sym_db.RegisterMessage(Components.PcollectionsEntry) -_sym_db.RegisterMessage(Components.WindowingStrategiesEntry) -_sym_db.RegisterMessage(Components.CodersEntry) -_sym_db.RegisterMessage(Components.EnvironmentsEntry) - -MessageWithComponents = _reflection.GeneratedProtocolMessageType('MessageWithComponents', (_message.Message,), dict( - DESCRIPTOR = _MESSAGEWITHCOMPONENTS, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.MessageWithComponents) - )) -_sym_db.RegisterMessage(MessageWithComponents) - -Pipeline = _reflection.GeneratedProtocolMessageType('Pipeline', (_message.Message,), dict( - DESCRIPTOR = _PIPELINE, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Pipeline) - )) -_sym_db.RegisterMessage(Pipeline) - -PTransform = _reflection.GeneratedProtocolMessageType('PTransform', (_message.Message,), dict( - - InputsEntry = _reflection.GeneratedProtocolMessageType('InputsEntry', (_message.Message,), dict( - DESCRIPTOR = _PTRANSFORM_INPUTSENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.PTransform.InputsEntry) - )) - , - - OutputsEntry = _reflection.GeneratedProtocolMessageType('OutputsEntry', (_message.Message,), dict( - DESCRIPTOR = _PTRANSFORM_OUTPUTSENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.PTransform.OutputsEntry) - )) - , - DESCRIPTOR = _PTRANSFORM, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.PTransform) - )) -_sym_db.RegisterMessage(PTransform) -_sym_db.RegisterMessage(PTransform.InputsEntry) -_sym_db.RegisterMessage(PTransform.OutputsEntry) - -PCollection = _reflection.GeneratedProtocolMessageType('PCollection', (_message.Message,), dict( - DESCRIPTOR = _PCOLLECTION, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.PCollection) - )) -_sym_db.RegisterMessage(PCollection) - -ParDoPayload = _reflection.GeneratedProtocolMessageType('ParDoPayload', (_message.Message,), dict( - - SideInputsEntry = _reflection.GeneratedProtocolMessageType('SideInputsEntry', (_message.Message,), dict( - DESCRIPTOR = _PARDOPAYLOAD_SIDEINPUTSENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.ParDoPayload.SideInputsEntry) - )) - , - - StateSpecsEntry = _reflection.GeneratedProtocolMessageType('StateSpecsEntry', (_message.Message,), dict( - DESCRIPTOR = _PARDOPAYLOAD_STATESPECSENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.ParDoPayload.StateSpecsEntry) - )) - , - - TimerSpecsEntry = _reflection.GeneratedProtocolMessageType('TimerSpecsEntry', (_message.Message,), dict( - DESCRIPTOR = _PARDOPAYLOAD_TIMERSPECSENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.ParDoPayload.TimerSpecsEntry) - )) - , - DESCRIPTOR = _PARDOPAYLOAD, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.ParDoPayload) - )) -_sym_db.RegisterMessage(ParDoPayload) -_sym_db.RegisterMessage(ParDoPayload.SideInputsEntry) -_sym_db.RegisterMessage(ParDoPayload.StateSpecsEntry) -_sym_db.RegisterMessage(ParDoPayload.TimerSpecsEntry) - -Parameter = _reflection.GeneratedProtocolMessageType('Parameter', (_message.Message,), dict( - DESCRIPTOR = _PARAMETER, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Parameter) - )) -_sym_db.RegisterMessage(Parameter) - -StateSpec = _reflection.GeneratedProtocolMessageType('StateSpec', (_message.Message,), dict( - DESCRIPTOR = _STATESPEC, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.StateSpec) - )) -_sym_db.RegisterMessage(StateSpec) - -TimerSpec = _reflection.GeneratedProtocolMessageType('TimerSpec', (_message.Message,), dict( - DESCRIPTOR = _TIMERSPEC, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.TimerSpec) - )) -_sym_db.RegisterMessage(TimerSpec) - -ReadPayload = _reflection.GeneratedProtocolMessageType('ReadPayload', (_message.Message,), dict( - DESCRIPTOR = _READPAYLOAD, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.ReadPayload) - )) -_sym_db.RegisterMessage(ReadPayload) - -WindowIntoPayload = _reflection.GeneratedProtocolMessageType('WindowIntoPayload', (_message.Message,), dict( - DESCRIPTOR = _WINDOWINTOPAYLOAD, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.WindowIntoPayload) - )) -_sym_db.RegisterMessage(WindowIntoPayload) - -CombinePayload = _reflection.GeneratedProtocolMessageType('CombinePayload', (_message.Message,), dict( - - SideInputsEntry = _reflection.GeneratedProtocolMessageType('SideInputsEntry', (_message.Message,), dict( - DESCRIPTOR = _COMBINEPAYLOAD_SIDEINPUTSENTRY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.CombinePayload.SideInputsEntry) - )) - , - DESCRIPTOR = _COMBINEPAYLOAD, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.CombinePayload) - )) -_sym_db.RegisterMessage(CombinePayload) -_sym_db.RegisterMessage(CombinePayload.SideInputsEntry) - -Coder = _reflection.GeneratedProtocolMessageType('Coder', (_message.Message,), dict( - DESCRIPTOR = _CODER, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Coder) - )) -_sym_db.RegisterMessage(Coder) - -WindowingStrategy = _reflection.GeneratedProtocolMessageType('WindowingStrategy', (_message.Message,), dict( - DESCRIPTOR = _WINDOWINGSTRATEGY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.WindowingStrategy) - )) -_sym_db.RegisterMessage(WindowingStrategy) - -Trigger = _reflection.GeneratedProtocolMessageType('Trigger', (_message.Message,), dict( - - AfterAll = _reflection.GeneratedProtocolMessageType('AfterAll', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_AFTERALL, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.AfterAll) - )) - , - - AfterAny = _reflection.GeneratedProtocolMessageType('AfterAny', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_AFTERANY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.AfterAny) - )) - , - - AfterEach = _reflection.GeneratedProtocolMessageType('AfterEach', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_AFTEREACH, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.AfterEach) - )) - , - - AfterEndOfWindow = _reflection.GeneratedProtocolMessageType('AfterEndOfWindow', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_AFTERENDOFWINDOW, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.AfterEndOfWindow) - )) - , - - AfterProcessingTime = _reflection.GeneratedProtocolMessageType('AfterProcessingTime', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_AFTERPROCESSINGTIME, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.AfterProcessingTime) - )) - , - - AfterSynchronizedProcessingTime = _reflection.GeneratedProtocolMessageType('AfterSynchronizedProcessingTime', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_AFTERSYNCHRONIZEDPROCESSINGTIME, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.AfterSynchronizedProcessingTime) - )) - , - - Default = _reflection.GeneratedProtocolMessageType('Default', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_DEFAULT, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.Default) - )) - , - - ElementCount = _reflection.GeneratedProtocolMessageType('ElementCount', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_ELEMENTCOUNT, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.ElementCount) - )) - , - - Never = _reflection.GeneratedProtocolMessageType('Never', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_NEVER, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.Never) - )) - , - - Always = _reflection.GeneratedProtocolMessageType('Always', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_ALWAYS, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.Always) - )) - , - - OrFinally = _reflection.GeneratedProtocolMessageType('OrFinally', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_ORFINALLY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.OrFinally) - )) - , - - Repeat = _reflection.GeneratedProtocolMessageType('Repeat', (_message.Message,), dict( - DESCRIPTOR = _TRIGGER_REPEAT, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger.Repeat) - )) - , - DESCRIPTOR = _TRIGGER, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Trigger) - )) -_sym_db.RegisterMessage(Trigger) -_sym_db.RegisterMessage(Trigger.AfterAll) -_sym_db.RegisterMessage(Trigger.AfterAny) -_sym_db.RegisterMessage(Trigger.AfterEach) -_sym_db.RegisterMessage(Trigger.AfterEndOfWindow) -_sym_db.RegisterMessage(Trigger.AfterProcessingTime) -_sym_db.RegisterMessage(Trigger.AfterSynchronizedProcessingTime) -_sym_db.RegisterMessage(Trigger.Default) -_sym_db.RegisterMessage(Trigger.ElementCount) -_sym_db.RegisterMessage(Trigger.Never) -_sym_db.RegisterMessage(Trigger.Always) -_sym_db.RegisterMessage(Trigger.OrFinally) -_sym_db.RegisterMessage(Trigger.Repeat) - -TimestampTransform = _reflection.GeneratedProtocolMessageType('TimestampTransform', (_message.Message,), dict( - - Delay = _reflection.GeneratedProtocolMessageType('Delay', (_message.Message,), dict( - DESCRIPTOR = _TIMESTAMPTRANSFORM_DELAY, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.TimestampTransform.Delay) - )) - , - - AlignTo = _reflection.GeneratedProtocolMessageType('AlignTo', (_message.Message,), dict( - DESCRIPTOR = _TIMESTAMPTRANSFORM_ALIGNTO, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.TimestampTransform.AlignTo) - )) - , - DESCRIPTOR = _TIMESTAMPTRANSFORM, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.TimestampTransform) - )) -_sym_db.RegisterMessage(TimestampTransform) -_sym_db.RegisterMessage(TimestampTransform.Delay) -_sym_db.RegisterMessage(TimestampTransform.AlignTo) - -SideInput = _reflection.GeneratedProtocolMessageType('SideInput', (_message.Message,), dict( - DESCRIPTOR = _SIDEINPUT, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.SideInput) - )) -_sym_db.RegisterMessage(SideInput) - -Environment = _reflection.GeneratedProtocolMessageType('Environment', (_message.Message,), dict( - DESCRIPTOR = _ENVIRONMENT, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.Environment) - )) -_sym_db.RegisterMessage(Environment) - -SdkFunctionSpec = _reflection.GeneratedProtocolMessageType('SdkFunctionSpec', (_message.Message,), dict( - DESCRIPTOR = _SDKFUNCTIONSPEC, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.SdkFunctionSpec) - )) -_sym_db.RegisterMessage(SdkFunctionSpec) - -FunctionSpec = _reflection.GeneratedProtocolMessageType('FunctionSpec', (_message.Message,), dict( - DESCRIPTOR = _FUNCTIONSPEC, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.FunctionSpec) - )) -_sym_db.RegisterMessage(FunctionSpec) - -DisplayData = _reflection.GeneratedProtocolMessageType('DisplayData', (_message.Message,), dict( - - Identifier = _reflection.GeneratedProtocolMessageType('Identifier', (_message.Message,), dict( - DESCRIPTOR = _DISPLAYDATA_IDENTIFIER, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.DisplayData.Identifier) - )) - , - - Item = _reflection.GeneratedProtocolMessageType('Item', (_message.Message,), dict( - DESCRIPTOR = _DISPLAYDATA_ITEM, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.DisplayData.Item) - )) - , - DESCRIPTOR = _DISPLAYDATA, - __module__ = 'beam_runner_api_pb2' - # @@protoc_insertion_point(class_scope:org.apache.beam.runner_api.v1.DisplayData) - )) -_sym_db.RegisterMessage(DisplayData) -_sym_db.RegisterMessage(DisplayData.Identifier) -_sym_db.RegisterMessage(DisplayData.Item) - - -DESCRIPTOR.has_options = True -DESCRIPTOR._options = _descriptor._ParseOptions(descriptor_pb2.FileOptions(), _b('\n$org.apache.beam.sdk.common.runner.v1B\tRunnerApi')) -_COMPONENTS_TRANSFORMSENTRY.has_options = True -_COMPONENTS_TRANSFORMSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_COMPONENTS_PCOLLECTIONSENTRY.has_options = True -_COMPONENTS_PCOLLECTIONSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_COMPONENTS_WINDOWINGSTRATEGIESENTRY.has_options = True -_COMPONENTS_WINDOWINGSTRATEGIESENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_COMPONENTS_CODERSENTRY.has_options = True -_COMPONENTS_CODERSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_COMPONENTS_ENVIRONMENTSENTRY.has_options = True -_COMPONENTS_ENVIRONMENTSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_PTRANSFORM_INPUTSENTRY.has_options = True -_PTRANSFORM_INPUTSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_PTRANSFORM_OUTPUTSENTRY.has_options = True -_PTRANSFORM_OUTPUTSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_PARDOPAYLOAD_SIDEINPUTSENTRY.has_options = True -_PARDOPAYLOAD_SIDEINPUTSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_PARDOPAYLOAD_STATESPECSENTRY.has_options = True -_PARDOPAYLOAD_STATESPECSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_PARDOPAYLOAD_TIMERSPECSENTRY.has_options = True -_PARDOPAYLOAD_TIMERSPECSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_COMBINEPAYLOAD_SIDEINPUTSENTRY.has_options = True -_COMBINEPAYLOAD_SIDEINPUTSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -# @@protoc_insertion_point(module_scope) From e9cb40dc37683effb2ded1bfaab6dfc04ade698b Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 17 May 2017 14:25:08 -0700 Subject: [PATCH 075/159] A couple of worker fixes. --- .../apache_beam/runners/worker/operation_specs.py | 11 ++++++----- sdks/python/apache_beam/runners/worker/operations.py | 2 +- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/runners/worker/operation_specs.py b/sdks/python/apache_beam/runners/worker/operation_specs.py index c03d9a25ec5f..db5eb765598b 100644 --- a/sdks/python/apache_beam/runners/worker/operation_specs.py +++ b/sdks/python/apache_beam/runners/worker/operation_specs.py @@ -305,7 +305,8 @@ def get_coder_from_spec(coder_spec): assert coder_spec is not None # Ignore the wrappers in these encodings. - # TODO(silviuc): Make sure with all the renamings that names below are ok. + ignored_wrappers = ( + 'com.google.cloud.dataflow.sdk.util.TimerOrElement$TimerOrElementCoder') if coder_spec['@type'] in ignored_wrappers: assert len(coder_spec['component_encodings']) == 1 coder_spec = coder_spec['component_encodings'][0] @@ -328,20 +329,20 @@ def get_coder_from_spec(coder_spec): return coders.WindowedValueCoder(value_coder, window_coder=window_coder) elif coder_spec['@type'] == 'kind:interval_window': assert ('component_encodings' not in coder_spec - or len(coder_spec['component_encodings'] == 0)) + or not coder_spec['component_encodings']) return coders.IntervalWindowCoder() elif coder_spec['@type'] == 'kind:global_window': assert ('component_encodings' not in coder_spec or not coder_spec['component_encodings']) - return coders.GlobalWindowCoder() + return coders.coders.GlobalWindowCoder() elif coder_spec['@type'] == 'kind:length_prefix': assert len(coder_spec['component_encodings']) == 1 - return coders.LengthPrefixCoder( + return coders.coders.LengthPrefixCoder( get_coder_from_spec(coder_spec['component_encodings'][0])) # We pass coders in the form "$" to make the job # description JSON more readable. - return coders.deserialize_coder(coder_spec['@type']) + return coders.coders.deserialize_coder(coder_spec['@type']) class MapTask(object): diff --git a/sdks/python/apache_beam/runners/worker/operations.py b/sdks/python/apache_beam/runners/worker/operations.py index 5dbe57e77b49..a44561d096c8 100644 --- a/sdks/python/apache_beam/runners/worker/operations.py +++ b/sdks/python/apache_beam/runners/worker/operations.py @@ -281,7 +281,7 @@ def _read_side_inputs(self, tags_and_types): # Backwards compatibility for pre BEAM-733 SDKs. if isinstance(view_options, tuple): - if view_class == pvalue.SingletonPCollectionView: + if view_class == pvalue.AsSingleton: has_default, default = view_options view_options = {'default': default} if has_default else {} else: From f32cacb7313b8352ddc054cd1b88d7a0462550db Mon Sep 17 00:00:00 2001 From: Pablo Date: Mon, 22 May 2017 15:39:44 -0700 Subject: [PATCH 076/159] Adding a snippet for metrics --- .../examples/snippets/snippets_test.py | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) diff --git a/sdks/python/apache_beam/examples/snippets/snippets_test.py b/sdks/python/apache_beam/examples/snippets/snippets_test.py index 6654fef654cb..e302465985a4 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets_test.py +++ b/sdks/python/apache_beam/examples/snippets/snippets_test.py @@ -32,6 +32,8 @@ from apache_beam.coders.coders import ToStringCoder from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.examples.snippets import snippets +from apache_beam.metrics import Metrics +from apache_beam.metrics.metric import MetricsFilter from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.utils.windowed_value import WindowedValue @@ -689,6 +691,66 @@ def test_model_co_group_by_key_tuple(self): expect = ['a; a@example.com; x4312', 'b; b@example.com; x8452'] self.assertEqual(expect, self.get_output(result_path)) + def test_model_use_and_query_metrics(self): + """DebuggingWordCount example snippets.""" + + import re + + p = TestPipeline() # Use TestPipeline for testing. + words = p | beam.Create(['albert', 'sam', 'mark', 'sarah', + 'swati', 'daniel', 'andrea']) + + # pylint: disable=unused-variable + # [START metrics_usage_example] + class FilterTextFn(beam.DoFn): + """A DoFn that filters for a specific key based on a regex.""" + + def __init__(self, pattern): + self.pattern = pattern + # A custom metric can track values in your pipeline as it runs. Create + # custom metrics to count unmatched words, and know the distribution of + # word lengths in the input PCollection. + self.word_len_dist = Metrics.distribution(self.__class__, + 'word_len_dist') + self.unmatched_words = Metrics.counter(self.__class__, + 'unmatched_words') + + def process(self, element): + word = element + self.word_len_dist.update(len(word)) + if re.match(self.pattern, word): + yield element + else: + self.unmatched_words.inc() + + filtered_words = ( + words | 'FilterText' >> beam.ParDo(FilterTextFn('s.*'))) + # [END metrics_usage_example] + # pylint: enable=unused-variable + + # [START metrics_check_values_example] + result = p.run() + result.wait_until_finish() + + custom_distribution = result.metrics().query( + MetricsFilter().with_name('word_len_dist'))['distributions'] + custom_counter = result.metrics().query( + MetricsFilter().with_name('unmatched_words'))['counters'] + + if custom_distribution: + logging.info('The average word length was %d', + custom_distribution[0].committed.mean) + if custom_counter: + logging.info('There were %d words that did not match the filter.', + custom_counter[0].committed) + # [END metrics_check_values_example] + + # There should be 4 words that did not match + self.assertEqual(custom_counter[0].committed, 4) + # The shortest word is 3 characters, the longest is 6 + self.assertEqual(custom_distribution[0].committed.min, 3) + self.assertEqual(custom_distribution[0].committed.max, 6) + def test_model_join_using_side_inputs(self): name_list = ['a', 'b'] email_list = [['a', 'a@example.com'], ['b', 'b@example.com']] From 8b0e4bc4ca93519f014eb309b755009cc55030bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Wed, 24 May 2017 17:38:07 +0200 Subject: [PATCH 077/159] Update maven-dependency-plugin to version 3.0.1 --- pom.xml | 3 ++- runners/apex/pom.xml | 5 +++++ runners/spark/pom.xml | 4 ++++ 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4f833cdb8e49..7e06b5f12952 100644 --- a/pom.xml +++ b/pom.xml @@ -151,6 +151,7 @@ 2.20 2.20 3.6.1 + 3.0.1 1.4.0 3.0.2 3.0.2 @@ -1632,7 +1633,7 @@ org.apache.maven.plugins maven-dependency-plugin - 3.0.0 + ${maven-dependency-plugin.version} analyze-only diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index f1a8a6245a4f..4a36bec8ab2c 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -78,6 +78,11 @@ + + org.apache.beam + beam-sdks-common-runner-api + + org.apache.beam beam-sdks-java-core diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 2c8372b754ef..697f67aadaec 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -216,6 +216,10 @@ 2.10.5 provided + + org.apache.beam + beam-sdks-common-runner-api + org.apache.beam beam-sdks-java-core From 26c93681795dc9b98c3f00c2cd61c38b36656e0b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 19 May 2017 21:35:43 -0700 Subject: [PATCH 078/159] Make SdkComponents public for TransformPayloadTranslator --- .../apache/beam/runners/core/construction/SdkComponents.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java index b0f164f4bcbd..0d3ba6093c3e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java @@ -46,7 +46,7 @@ import org.apache.beam.sdk.values.WindowingStrategy; /** SDK objects that will be represented at some later point within a {@link Components} object. */ -class SdkComponents { +public class SdkComponents { private final RunnerApi.Components.Builder componentsBuilder; private final BiMap, String> transformIds; From 1dc134cf341d1eb4fa936f6fe7b83a3edbb64687 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 19 May 2017 20:54:36 -0700 Subject: [PATCH 079/159] Centralize primitive URNs in PTransformTranslation class --- .../runners/core/construction/PTransformTranslation.java | 7 +++++++ .../beam/runners/core/construction/ParDoTranslation.java | 9 +++------ 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index 86638dec8098..8be023ad413b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -38,6 +38,13 @@ * protocol buffers}. */ public class PTransformTranslation { + + public static final String PAR_DO_TRANSFORM_URN = "urn:beam:transform:pardo:v1"; + public static final String FLATTEN_TRANSFORM_URN = "urn:beam:transform:flatten:v1"; + public static final String GROUP_BY_KEY_TRANSFORM_URN = "urn:beam:transform:groupbykey:v1"; + public static final String READ_TRANSFORM_URN = "urn:beam:transform:read:v1"; + public static final String WINDOW_TRANSFORM_URN = "urn:beam:transform:window:v1"; + private static final Map, TransformPayloadTranslator> KNOWN_PAYLOAD_TRANSLATORS = loadTransformPayloadTranslators(); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java index 28d577ff2211..83277bbefd65 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java @@ -19,6 +19,7 @@ package org.apache.beam.runners.core.construction; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; @@ -74,10 +75,6 @@ * Utilities for interacting with {@link ParDo} instances and {@link ParDoPayload} protos. */ public class ParDoTranslation { - /** - * The URN for a {@link ParDoPayload}. - */ - public static final String PAR_DO_PAYLOAD_URN = "urn:beam:pardo:v1"; /** * The URN for an unknown Java {@link DoFn}. */ @@ -108,7 +105,7 @@ public FunctionSpec translate( AppliedPTransform> transform, SdkComponents components) { ParDoPayload payload = toProto(transform.getTransform(), components); return RunnerApi.FunctionSpec.newBuilder() - .setUrn(PAR_DO_PAYLOAD_URN) + .setUrn(PAR_DO_TRANSFORM_URN) .setParameter(Any.pack(payload)) .build(); } @@ -166,7 +163,7 @@ public static TupleTag getMainOutputTag(ParDoPayload payload) public static RunnerApi.PCollection getMainInput( RunnerApi.PTransform ptransform, Components components) throws IOException { checkArgument( - ptransform.getSpec().getUrn().equals(PAR_DO_PAYLOAD_URN), + ptransform.getSpec().getUrn().equals(PAR_DO_TRANSFORM_URN), "Unexpected payload type %s", ptransform.getSpec().getUrn()); ParDoPayload payload = ptransform.getSpec().getParameter().unpack(ParDoPayload.class); From 877408e078b783b3a782ae1ba0e335ea44c4a0e2 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 24 May 2017 15:28:38 -0700 Subject: [PATCH 080/159] Add URN for Splittable ProcessElement pseudo-primitive --- .../apache/beam/runners/core/construction/SplittableParDo.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java index 23ba66ab983d..dfca7d2d94eb 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java @@ -64,6 +64,9 @@ public class SplittableParDo extends PTransform, PCollectionTuple> { private final ParDo.MultiOutput parDo; + public static final String SPLITTABLE_PROCESS_URN = + "urn:beam:runners_core:transforms:splittable_process:v1"; + /** * Creates the transform for the given original multi-output {@link ParDo}. * From 0bf4ddbea7d0f790fad0cc8df20f0d01f38dd568 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 19 May 2017 20:55:25 -0700 Subject: [PATCH 081/159] Allow getting URN for class of transform via translator --- .../core/construction/PTransformTranslation.java | 14 +++++++++++++- .../core/construction/ParDoTranslation.java | 5 +++++ .../core/construction/WindowIntoTranslation.java | 7 +++++++ 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index 8be023ad413b..35bb0e3dad4e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -116,11 +116,23 @@ private static String toProto(TupleTag tag) { return tag.getId(); } + public static String urnForTransform(PTransform transform) { + TransformPayloadTranslator translator = + KNOWN_PAYLOAD_TRANSLATORS.get(transform.getClass()); + if (translator == null) { + throw new IllegalStateException( + String.format("No translator known for %s", transform.getClass().getName())); + } + + return translator.getUrn(transform); + } + /** * A translator consumes a {@link PTransform} application and produces the appropriate * FunctionSpec for a distinguished or primitive transform within the Beam runner API. */ public interface TransformPayloadTranslator> { - FunctionSpec translate(AppliedPTransform transform, SdkComponents components); + String getUrn(T transform); + FunctionSpec translate(AppliedPTransform application, SdkComponents components); } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java index 83277bbefd65..1c81f8ce05d5 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java @@ -100,6 +100,11 @@ public static TransformPayloadTranslator create() { private ParDoPayloadTranslator() {} + @Override + public String getUrn(ParDo.MultiOutput transform) { + return PAR_DO_TRANSFORM_URN; + } + @Override public FunctionSpec translate( AppliedPTransform> transform, SdkComponents components) { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java index 215beba43cd8..33faa02f8801 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.common.runner.v1.RunnerApi.WindowIntoPayload; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.Window.Assign; import org.apache.beam.sdk.transforms.windowing.WindowFn; /** @@ -36,6 +37,12 @@ public class WindowIntoTranslation { static class WindowAssignTranslator implements TransformPayloadTranslator> { + + @Override + public String getUrn(Assign transform) { + return PTransforms.WINDOW_TRANSFORM_URN; + } + @Override public FunctionSpec translate( AppliedPTransform> transform, SdkComponents components) { From afaebb13f73868799b83c7af95cca2732e3ecb9a Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 21:00:02 -0700 Subject: [PATCH 082/159] Add registration for Read and WindowInto translators --- .../core/construction/ReadTranslation.java | 74 ++++++++++++++++++- .../construction/WindowIntoTranslation.java | 43 ++++++++++- 2 files changed, 115 insertions(+), 2 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java index d6c34008f29e..aff5fc9686f7 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java @@ -20,10 +20,15 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableMap; import com.google.protobuf.Any; import com.google.protobuf.ByteString; import com.google.protobuf.BytesValue; import com.google.protobuf.InvalidProtocolBufferException; +import java.util.Map; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.common.runner.v1.RunnerApi.IsBounded; import org.apache.beam.sdk.common.runner.v1.RunnerApi.ReadPayload; @@ -32,12 +37,13 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.SerializableUtils; /** * Methods for translating {@link Read.Bounded} and {@link Read.Unbounded} - * {@link PTransform PTransforms} into {@link ReadPayload} protos. + * {@link PTransform PTransformTranslation} into {@link ReadPayload} protos. */ public class ReadTranslation { private static final String JAVA_SERIALIZED_BOUNDED_SOURCE = "urn:beam:java:boundedsource:v1"; @@ -124,4 +130,70 @@ private static SdkFunctionSpec toProto(UnboundedSource source) { "BoundedSource"); } + /** + * A {@link TransformPayloadTranslator} for {@link Read.Unbounded}. + */ + public static class UnboundedReadPayloadTranslator + implements PTransformTranslation.TransformPayloadTranslator> { + public static TransformPayloadTranslator create() { + return new UnboundedReadPayloadTranslator(); + } + + private UnboundedReadPayloadTranslator() {} + + @Override + public String getUrn(Read.Unbounded transform) { + return PTransformTranslation.WINDOW_TRANSFORM_URN; + } + + @Override + public FunctionSpec translate( + AppliedPTransform> transform, SdkComponents components) { + ReadPayload payload = toProto(transform.getTransform()); + return RunnerApi.FunctionSpec.newBuilder() + .setUrn(PTransformTranslation.READ_TRANSFORM_URN) + .setParameter(Any.pack(payload)) + .build(); + } + } + + /** + * A {@link TransformPayloadTranslator} for {@link Read.Bounded}. + */ + public static class BoundedReadPayloadTranslator + implements PTransformTranslation.TransformPayloadTranslator> { + public static TransformPayloadTranslator create() { + return new BoundedReadPayloadTranslator(); + } + + private BoundedReadPayloadTranslator() {} + + @Override + public String getUrn(Read.Bounded transform) { + return PTransformTranslation.WINDOW_TRANSFORM_URN; + } + + @Override + public FunctionSpec translate( + AppliedPTransform> transform, SdkComponents components) { + ReadPayload payload = toProto(transform.getTransform()); + return RunnerApi.FunctionSpec.newBuilder() + .setUrn(PTransformTranslation.READ_TRANSFORM_URN) + .setParameter(Any.pack(payload)) + .build(); + } + } + + /** Registers {@link UnboundedReadPayloadTranslator} and {@link BoundedReadPayloadTranslator}. */ + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class Registrar implements TransformPayloadTranslatorRegistrar { + @Override + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return ImmutableMap., TransformPayloadTranslator>builder() + .put(Read.Unbounded.class, new UnboundedReadPayloadTranslator()) + .put(Read.Bounded.class, new BoundedReadPayloadTranslator()) + .build(); + } + } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java index 33faa02f8801..5ed4d24906b1 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java @@ -18,14 +18,18 @@ package org.apache.beam.runners.core.construction; +import com.google.auto.service.AutoService; import com.google.protobuf.Any; import com.google.protobuf.InvalidProtocolBufferException; +import java.util.Collections; +import java.util.Map; import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; import org.apache.beam.sdk.common.runner.v1.RunnerApi.WindowIntoPayload; import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.Window.Assign; import org.apache.beam.sdk.transforms.windowing.WindowFn; @@ -40,7 +44,7 @@ static class WindowAssignTranslator implements TransformPayloadTranslator transform) { - return PTransforms.WINDOW_TRANSFORM_URN; + return PTransformTranslation.WINDOW_TRANSFORM_URN; } @Override @@ -65,4 +69,41 @@ public static WindowIntoPayload toProto(Window.Assign transform, SdkComponent SdkFunctionSpec spec = payload.getWindowFn(); return WindowingStrategyTranslation.windowFnFromProto(spec); } + + /** + * A {@link TransformPayloadTranslator} for {@link Window}. + */ + public static class WindowIntoPayloadTranslator + implements PTransformTranslation.TransformPayloadTranslator> { + public static TransformPayloadTranslator create() { + return new WindowIntoPayloadTranslator(); + } + + private WindowIntoPayloadTranslator() {} + + @Override + public String getUrn(Window.Assign transform) { + return PTransformTranslation.WINDOW_TRANSFORM_URN; + } + + @Override + public FunctionSpec translate( + AppliedPTransform> transform, SdkComponents components) { + WindowIntoPayload payload = toProto(transform.getTransform(), components); + return RunnerApi.FunctionSpec.newBuilder() + .setUrn(PTransformTranslation.WINDOW_TRANSFORM_URN) + .setParameter(Any.pack(payload)) + .build(); + } + } + + /** Registers {@link WindowIntoPayloadTranslator}. */ + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class Registrar implements TransformPayloadTranslatorRegistrar { + @Override + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return Collections.singletonMap(Window.Assign.class, new WindowIntoPayloadTranslator()); + } + } } From d91c840f009bdfda14f8668172d076a7e9f12f5e Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 21:09:20 -0700 Subject: [PATCH 083/159] Register ReadTranslator --- .../beam/runners/core/construction/ReadTranslation.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java index aff5fc9686f7..3ddde8d767aa 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java @@ -143,7 +143,7 @@ private UnboundedReadPayloadTranslator() {} @Override public String getUrn(Read.Unbounded transform) { - return PTransformTranslation.WINDOW_TRANSFORM_URN; + return PTransformTranslation.READ_TRANSFORM_URN; } @Override @@ -151,7 +151,7 @@ public FunctionSpec translate( AppliedPTransform> transform, SdkComponents components) { ReadPayload payload = toProto(transform.getTransform()); return RunnerApi.FunctionSpec.newBuilder() - .setUrn(PTransformTranslation.READ_TRANSFORM_URN) + .setUrn(getUrn(transform.getTransform())) .setParameter(Any.pack(payload)) .build(); } @@ -170,7 +170,7 @@ private BoundedReadPayloadTranslator() {} @Override public String getUrn(Read.Bounded transform) { - return PTransformTranslation.WINDOW_TRANSFORM_URN; + return PTransformTranslation.READ_TRANSFORM_URN; } @Override @@ -178,7 +178,7 @@ public FunctionSpec translate( AppliedPTransform> transform, SdkComponents components) { ReadPayload payload = toProto(transform.getTransform()); return RunnerApi.FunctionSpec.newBuilder() - .setUrn(PTransformTranslation.READ_TRANSFORM_URN) + .setUrn(getUrn(transform.getTransform())) .setParameter(Any.pack(payload)) .build(); } From 69bb1be4d3bd20e5c080bcce0eb464fac3d2ff73 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 21:09:33 -0700 Subject: [PATCH 084/159] Register WindowIntoTranslator --- .../beam/runners/core/construction/WindowIntoTranslation.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java index 5ed4d24906b1..aa17bc90d6a3 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java @@ -91,7 +91,7 @@ public FunctionSpec translate( AppliedPTransform> transform, SdkComponents components) { WindowIntoPayload payload = toProto(transform.getTransform(), components); return RunnerApi.FunctionSpec.newBuilder() - .setUrn(PTransformTranslation.WINDOW_TRANSFORM_URN) + .setUrn(getUrn(transform.getTransform())) .setParameter(Any.pack(payload)) .build(); } From 121631c62f94a6e553d3bbd8708cee2dbdf53923 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 May 2017 21:09:54 -0700 Subject: [PATCH 085/159] Add trivial FlattenTranslator to access URN --- .../core/construction/FlattenTranslator.java | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java new file mode 100644 index 000000000000..f1d553dc8abd --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/FlattenTranslator.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import com.google.auto.service.AutoService; +import java.util.Collections; +import java.util.Map; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.windowing.Window.Assign; + +/** + * Utility methods for translating a {@link Assign} to and from {@link RunnerApi} representations. + */ +public class FlattenTranslator implements TransformPayloadTranslator> { + + public static TransformPayloadTranslator create() { + return new FlattenTranslator(); + } + + private FlattenTranslator() {} + + @Override + public String getUrn(Flatten.PCollections transform) { + return PTransformTranslation.FLATTEN_TRANSFORM_URN; + } + + @Override + public FunctionSpec translate( + AppliedPTransform> transform, SdkComponents components) { + return RunnerApi.FunctionSpec.newBuilder().setUrn(getUrn(transform.getTransform())).build(); + } + + /** Registers {@link FlattenTranslator}. */ + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class Registrar implements TransformPayloadTranslatorRegistrar { + @Override + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return Collections.singletonMap(Flatten.PCollections.class, new FlattenTranslator()); + } + } +} From 8e09596a981fe69edb7a1560e864bc852978ba81 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 23 May 2017 21:15:48 -0700 Subject: [PATCH 086/159] Add transform-analysis helpers to ReadTranslation These helpers allow a runner to extract a source and inspect boundedness without reference to the specific user-facing Java classes Read.Bounded and Read.Unbounded. --- .../construction/PCollectionTranslation.java | 4 +- .../core/construction/ReadTranslation.java | 43 +++++++++++++++++++ 2 files changed, 45 insertions(+), 2 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java index 303c02d44a11..968966f459e1 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java @@ -64,7 +64,7 @@ public static Coder getCoder( components.getWindowingStrategiesOrThrow(pCollection.getWindowingStrategyId()), components); } - private static RunnerApi.IsBounded toProto(IsBounded bounded) { + static RunnerApi.IsBounded toProto(IsBounded bounded) { switch (bounded) { case BOUNDED: return RunnerApi.IsBounded.BOUNDED; @@ -76,7 +76,7 @@ private static RunnerApi.IsBounded toProto(IsBounded bounded) { } } - private static IsBounded fromProto(RunnerApi.IsBounded isBounded) { + static IsBounded fromProto(RunnerApi.IsBounded isBounded) { switch (isBounded) { case BOUNDED: return IsBounded.BOUNDED; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java index 3ddde8d767aa..572384bdd549 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java @@ -26,6 +26,8 @@ import com.google.protobuf.ByteString; import com.google.protobuf.BytesValue; import com.google.protobuf.InvalidProtocolBufferException; +import java.io.IOException; +import java.util.Collections; import java.util.Map; import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.common.runner.v1.RunnerApi; @@ -40,6 +42,8 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; /** * Methods for translating {@link Read.Bounded} and {@link Read.Unbounded} @@ -102,6 +106,29 @@ public static BoundedSource boundedSourceFromProto(ReadPayload payload) "BoundedSource"); } + public static BoundedSource boundedSourceFromTransform( + AppliedPTransform, PTransform>> transform) + throws IOException { + return (BoundedSource) boundedSourceFromProto(getReadPayload(transform)); + } + + public static + UnboundedSource unboundedSourceFromTransform( + AppliedPTransform, PTransform>> transform) + throws IOException { + return (UnboundedSource) unboundedSourceFromProto(getReadPayload(transform)); + } + + private static ReadPayload getReadPayload( + AppliedPTransform, PTransform>> transform) + throws IOException { + return PTransformTranslation.toProto( + transform, Collections.>emptyList(), SdkComponents.create()) + .getSpec() + .getParameter() + .unpack(ReadPayload.class); + } + private static SdkFunctionSpec toProto(UnboundedSource source) { return SdkFunctionSpec.newBuilder() .setSpec( @@ -130,6 +157,22 @@ private static SdkFunctionSpec toProto(UnboundedSource source) { "BoundedSource"); } + public static PCollection.IsBounded sourceIsBounded(AppliedPTransform transform) { + try { + return PCollectionTranslation.fromProto( + PTransformTranslation.toProto( + transform, + Collections.>emptyList(), + SdkComponents.create()) + .getSpec() + .getParameter() + .unpack(ReadPayload.class) + .getIsBounded()); + } catch (IOException e) { + throw new RuntimeException("Internal error determining boundedness of Read", e); + } + } + /** * A {@link TransformPayloadTranslator} for {@link Read.Unbounded}. */ From 663ad88178ceadffe4cfa592555986ed7dde58b4 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 19 May 2017 21:28:19 -0700 Subject: [PATCH 087/159] Add RawPTransform, which can just vend its URN and payload This is the type that will be returned when a pipeline is deserialized. This also is convenient for direct runner overrides which do not really merit translator registrations, yet URNs need to be known in order to key the evaluator registry off URN. --- .../construction/PTransformTranslation.java | 51 +++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index 35bb0e3dad4e..9f5f3b50b723 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -21,15 +21,20 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.common.collect.ImmutableMap; +import com.google.protobuf.Any; +import com.google.protobuf.Message; import java.io.IOException; import java.util.List; import java.util.Map; import java.util.ServiceLoader; +import javax.annotation.Nullable; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; @@ -135,4 +140,50 @@ public interface TransformPayloadTranslator> { String getUrn(T transform); FunctionSpec translate(AppliedPTransform application, SdkComponents components); } + + /** + * A {@link PTransform} that indicates its URN and payload directly. + * + *

This is the result of rehydrating transforms from a pipeline proto. There is no {@link + * #expand} method since the definition of the transform may be lost. The transform is already + * fully expanded in the pipeline proto. + */ + public abstract static class RawPTransform< + InputT extends PInput, OutputT extends POutput, PayloadT extends Message> + extends PTransform { + + public abstract String getUrn(); + + @Nullable + PayloadT getPayload() { + return null; + } + } + + /** + * A translator that uses the explicit URN and payload from a {@link RawPTransform}. + */ + public static class RawPTransformTranslator + implements TransformPayloadTranslator> { + @Override + public String getUrn(RawPTransform transform) { + return transform.getUrn(); + } + + @Override + public FunctionSpec translate( + AppliedPTransform> transform, + SdkComponents components) { + PayloadT payload = transform.getTransform().getPayload(); + + FunctionSpec.Builder transformSpec = + FunctionSpec.newBuilder().setUrn(getUrn(transform.getTransform())); + + if (payload != null) { + transformSpec.setParameter(Any.pack(payload)); + } + + return transformSpec.build(); + } + } } From 0e29cc52a3e4b0d9ae5ff3907f10e4e87b734186 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 19 May 2017 20:53:32 -0700 Subject: [PATCH 088/159] URNs for DirectRunner TransformEvaluator and RootInputProvider This makes two of the Java DirectRunner's registries key off URN instead of Java class. A root TransformEvaluator requires shards generated by its associated RootInputProvider, hence changing both at once. --- runners/direct-java/pom.xml | 5 + .../direct/BoundedReadEvaluatorFactory.java | 14 +- .../beam/runners/direct/DirectGroupByKey.java | 21 ++- .../runners/direct/EmptyInputProvider.java | 8 +- .../direct/ParDoMultiOverrideFactory.java | 13 +- .../runners/direct/ReadEvaluatorFactory.java | 97 +++++++++++++ .../runners/direct/RootInputProvider.java | 7 +- .../runners/direct/RootProviderRegistry.java | 28 ++-- .../beam/runners/direct/SourceShard.java | 33 +++++ .../direct/TestStreamEvaluatorFactory.java | 28 ++-- .../direct/TransformEvaluatorRegistry.java | 128 +++++++++++++----- .../direct/UnboundedReadEvaluatorFactory.java | 31 +++-- .../runners/direct/ViewOverrideFactory.java | 12 +- .../main/resources/beam/findbugs-filter.xml | 7 + 14 files changed, 344 insertions(+), 88 deletions(-) create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/ReadEvaluatorFactory.java create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/SourceShard.java diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index bec21139d989..cba4b099ff0f 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -207,6 +207,11 @@ test + + com.google.protobuf + protobuf-java + + joda-time joda-time diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java index 76db861897e0..fcaaa8442082 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java @@ -33,10 +33,10 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import javax.annotation.Nullable; +import org.apache.beam.runners.core.construction.ReadTranslation; import org.apache.beam.runners.direct.StepTransformResult.Builder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; -import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.Read.Bounded; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -180,16 +180,17 @@ public TransformResult> finishBundle() { } @AutoValue - abstract static class BoundedSourceShard { + abstract static class BoundedSourceShard implements SourceShard { static BoundedSourceShard of(BoundedSource source) { return new AutoValue_BoundedReadEvaluatorFactory_BoundedSourceShard<>(source); } - abstract BoundedSource getSource(); + @Override + public abstract BoundedSource getSource(); } static class InputProvider - implements RootInputProvider, PBegin, Read.Bounded> { + implements RootInputProvider, PBegin> { private final EvaluationContext evaluationContext; InputProvider(EvaluationContext evaluationContext) { @@ -198,9 +199,10 @@ static class InputProvider @Override public Collection>> getInitialInputs( - AppliedPTransform, Read.Bounded> transform, int targetParallelism) + AppliedPTransform, PTransform>> transform, + int targetParallelism) throws Exception { - BoundedSource source = transform.getTransform().getSource(); + BoundedSource source = ReadTranslation.boundedSourceFromTransform(transform); PipelineOptions options = evaluationContext.getPipelineOptions(); long estimatedBytes = source.getEstimatedSizeBytes(options); long bytesPerBundle = estimatedBytes / targetParallelism; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java index 791615a2a194..f239070d925b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKey.java @@ -20,9 +20,11 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.protobuf.Message; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItemCoder; import org.apache.beam.runners.core.construction.ForwardingPTransform; +import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; @@ -37,6 +39,9 @@ class DirectGroupByKey extends ForwardingPTransform>, PCollection>>> { private final GroupByKey original; + static final String DIRECT_GBKO_URN = "urn:beam:directrunner:transforms:gbko:v1"; + static final String DIRECT_GABW_URN = "urn:beam:directrunner:transforms:gabw:v1"; + DirectGroupByKey(GroupByKey from) { this.original = from; } @@ -68,7 +73,8 @@ public PCollection>> expand(PCollection> input) { } static final class DirectGroupByKeyOnly - extends PTransform>, PCollection>> { + extends PTransformTranslation.RawPTransform< + PCollection>, PCollection>, Message> { @Override public PCollection> expand(PCollection> input) { return PCollection.createPrimitiveOutputInternal( @@ -86,10 +92,16 @@ protected Coder getDefaultOutputCoder( GroupByKey.getInputValueCoder(input.getCoder()), input.getWindowingStrategy().getWindowFn().windowCoder()); } + + @Override + public String getUrn() { + return DIRECT_GBKO_URN; + } } static final class DirectGroupAlsoByWindow - extends PTransform>, PCollection>>> { + extends PTransformTranslation.RawPTransform< + PCollection>, PCollection>>, Message> { private final WindowingStrategy inputWindowingStrategy; private final WindowingStrategy outputWindowingStrategy; @@ -135,5 +147,10 @@ public PCollection>> expand(PCollection> i return PCollection.createPrimitiveOutputInternal( input.getPipeline(), outputWindowingStrategy, input.isBounded()); } + + @Override + public String getUrn() { + return DIRECT_GABW_URN; + } } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyInputProvider.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyInputProvider.java index c36879a5f7b6..a5a53bc8ea6c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyInputProvider.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EmptyInputProvider.java @@ -20,13 +20,12 @@ import java.util.Collection; import java.util.Collections; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; /** A {@link RootInputProvider} that provides a singleton empty bundle. */ -class EmptyInputProvider - implements RootInputProvider, Flatten.PCollections> { +class EmptyInputProvider implements RootInputProvider> { EmptyInputProvider() {} /** @@ -36,7 +35,8 @@ class EmptyInputProvider */ @Override public Collection> getInitialInputs( - AppliedPTransform, PCollection, Flatten.PCollections> + AppliedPTransform< + PCollectionList, PCollection, PTransform, PCollection>> transform, int targetParallelism) { return Collections.emptyList(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index be433db1ca00..df2054b333ab 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -19,11 +19,13 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.protobuf.Message; import java.util.Map; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItemCoder; import org.apache.beam.runners.core.KeyedWorkItems; import org.apache.beam.runners.core.construction.PTransformReplacements; +import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.runners.core.construction.SplittableParDo; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -165,8 +167,12 @@ public PCollectionTuple expand(PCollection> input) { } } + static final String DIRECT_STATEFUL_PAR_DO_URN = + "urn:beam:directrunner:transforms:stateful_pardo:v1"; + static class StatefulParDo - extends PTransform>>, PCollectionTuple> { + extends PTransformTranslation.RawPTransform< + PCollection>>, PCollectionTuple, Message> { private final transient MultiOutput, OutputT> underlyingParDo; private final transient PCollection> originalInput; @@ -201,6 +207,11 @@ public PCollectionTuple expand(PCollection TransformEvaluator forApplication( + AppliedPTransform application, CommittedBundle inputBundle) throws Exception { + switch (ReadTranslation.sourceIsBounded(application)) { + case BOUNDED: + return boundedFactory.forApplication(application, inputBundle); + case UNBOUNDED: + return unboundedFactory.forApplication(application, inputBundle); + default: + throw new IllegalArgumentException("PCollection is neither bounded nor unbounded?!?"); + } + } + + @Override + public void cleanup() throws Exception { + boundedFactory.cleanup(); + unboundedFactory.cleanup(); + } + + static InputProvider inputProvider(EvaluationContext context) { + return new InputProvider(context); + } + + private static class InputProvider implements RootInputProvider, PBegin> { + + private final UnboundedReadEvaluatorFactory.InputProvider unboundedInputProvider; + private final BoundedReadEvaluatorFactory.InputProvider boundedInputProvider; + + InputProvider(EvaluationContext context) { + this.unboundedInputProvider = new UnboundedReadEvaluatorFactory.InputProvider(context); + this.boundedInputProvider = new BoundedReadEvaluatorFactory.InputProvider(context); + } + + @Override + public Collection>> getInitialInputs( + AppliedPTransform, PTransform>> + appliedTransform, + int targetParallelism) + throws Exception { + switch (ReadTranslation.sourceIsBounded(appliedTransform)) { + case BOUNDED: + // This cast could be made unnecessary, but too much bounded polymorphism + return (Collection) + boundedInputProvider.getInitialInputs(appliedTransform, targetParallelism); + case UNBOUNDED: + // This cast could be made unnecessary, but too much bounded polymorphism + return (Collection) + unboundedInputProvider.getInitialInputs(appliedTransform, targetParallelism); + default: + throw new IllegalArgumentException("PCollection is neither bounded nor unbounded?!?"); + } + } + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootInputProvider.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootInputProvider.java index ce6951805117..0b3de3226aca 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootInputProvider.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootInputProvider.java @@ -29,8 +29,7 @@ * Provides {@link CommittedBundle bundles} that will be provided to the {@link PTransform * PTransforms} that are at the root of a {@link Pipeline}. */ -interface RootInputProvider< - T, ShardT, InputT extends PInput, TransformT extends PTransform>> { +interface RootInputProvider { /** * Get the initial inputs for the {@link AppliedPTransform}. The {@link AppliedPTransform} will be * provided with these {@link CommittedBundle bundles} as input when the {@link Pipeline} runs. @@ -44,6 +43,8 @@ interface RootInputProvider< * greater than or equal to 1. */ Collection> getInitialInputs( - AppliedPTransform, TransformT> transform, int targetParallelism) + AppliedPTransform, PTransform>> + transform, + int targetParallelism) throws Exception; } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java index 4b0c06d3611f..5cbeab7db249 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java @@ -18,13 +18,14 @@ package org.apache.beam.runners.direct; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.runners.core.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN; +import static org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory.DIRECT_TEST_STREAM_URN; import com.google.common.collect.ImmutableMap; import java.util.Collection; import java.util.Map; -import org.apache.beam.sdk.io.Read; +import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.transforms.Flatten.PCollections; import org.apache.beam.sdk.transforms.PTransform; /** @@ -33,34 +34,31 @@ */ class RootProviderRegistry { public static RootProviderRegistry defaultRegistry(EvaluationContext context) { - ImmutableMap.Builder, RootInputProvider> + ImmutableMap.Builder> defaultProviders = ImmutableMap.builder(); defaultProviders - .put(Read.Bounded.class, new BoundedReadEvaluatorFactory.InputProvider(context)) - .put(Read.Unbounded.class, new UnboundedReadEvaluatorFactory.InputProvider(context)) - .put( - TestStreamEvaluatorFactory.DirectTestStreamFactory.DirectTestStream.class, - new TestStreamEvaluatorFactory.InputProvider(context)) - .put(PCollections.class, new EmptyInputProvider()); + .put(PTransformTranslation.READ_TRANSFORM_URN, ReadEvaluatorFactory.inputProvider(context)) + .put(DIRECT_TEST_STREAM_URN, new TestStreamEvaluatorFactory.InputProvider(context)) + .put(FLATTEN_TRANSFORM_URN, new EmptyInputProvider()); return new RootProviderRegistry(defaultProviders.build()); } - private final Map, RootInputProvider> providers; + private final Map> providers; private RootProviderRegistry( - Map, RootInputProvider> providers) { + Map> providers) { this.providers = providers; } public Collection> getInitialInputs( AppliedPTransform transform, int targetParallelism) throws Exception { - Class transformClass = transform.getTransform().getClass(); + String transformUrn = PTransformTranslation.urnForTransform(transform.getTransform()); RootInputProvider provider = checkNotNull( - providers.get(transformClass), - "Tried to get a %s for a Transform of type %s, but there is no such provider", + providers.get(transformUrn), + "Tried to get a %s for a transform \"%s\", but there is no such provider", RootInputProvider.class.getSimpleName(), - transformClass.getSimpleName()); + transformUrn); return provider.getInitialInputs(transform, targetParallelism); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SourceShard.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SourceShard.java new file mode 100644 index 000000000000..a054333d70fc --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SourceShard.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.direct; + +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.Source; +import org.apache.beam.sdk.io.UnboundedSource; + +/** + * A shard for a source in the {@link Read} transform. + * + *

Since {@link UnboundedSource} and {@link BoundedSource} have radically different needs, this + * is a mostly-empty interface. + */ +interface SourceShard { + Source getSource(); +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java index 8b21d5affd60..b1db58f7c4a0 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -22,12 +22,14 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Supplier; import com.google.common.collect.Iterables; +import com.google.protobuf.Message; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; +import org.apache.beam.runners.core.construction.PTransformTranslation; import org.apache.beam.runners.core.construction.ReplacementOutputs; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; @@ -180,7 +182,10 @@ public Map mapOutputs( return ReplacementOutputs.singleton(outputs, newOutput); } - static class DirectTestStream extends PTransform> { + static final String DIRECT_TEST_STREAM_URN = "urn:beam:directrunner:transforms:test_stream:v1"; + + static class DirectTestStream + extends PTransformTranslation.RawPTransform, Message> { private final transient DirectRunner runner; private final TestStream original; @@ -197,12 +202,15 @@ public PCollection expand(PBegin input) { input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) .setCoder(original.getValueCoder()); } + + @Override + public String getUrn() { + return DIRECT_TEST_STREAM_URN; + } } } - static class InputProvider - implements RootInputProvider< - T, TestStreamIndex, PBegin, DirectTestStreamFactory.DirectTestStream> { + static class InputProvider implements RootInputProvider, PBegin> { private final EvaluationContext evaluationContext; InputProvider(EvaluationContext evaluationContext) { @@ -211,15 +219,17 @@ static class InputProvider @Override public Collection>> getInitialInputs( - AppliedPTransform, DirectTestStreamFactory.DirectTestStream> - transform, + AppliedPTransform, PTransform>> transform, int targetParallelism) { + + // This will always be run on an execution-time transform, so it can be downcast + DirectTestStreamFactory.DirectTestStream testStream = + (DirectTestStreamFactory.DirectTestStream) transform.getTransform(); + CommittedBundle> initialBundle = evaluationContext .>createRootBundle() - .add( - WindowedValue.valueInGlobalWindow( - TestStreamIndex.of(transform.getTransform().original))) + .add(WindowedValue.valueInGlobalWindow(TestStreamIndex.of(testStream.original))) .commit(BoundedWindow.TIMESTAMP_MAX_VALUE); return Collections.singleton(initialBundle); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java index 718cca2f5ddb..d144b2042d2b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java @@ -19,23 +19,33 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import static org.apache.beam.runners.core.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN; +import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; +import static org.apache.beam.runners.core.construction.PTransformTranslation.READ_TRANSFORM_URN; +import static org.apache.beam.runners.core.construction.PTransformTranslation.WINDOW_TRANSFORM_URN; +import static org.apache.beam.runners.core.construction.SplittableParDo.SPLITTABLE_PROCESS_URN; +import static org.apache.beam.runners.direct.DirectGroupByKey.DIRECT_GABW_URN; +import static org.apache.beam.runners.direct.DirectGroupByKey.DIRECT_GBKO_URN; +import static org.apache.beam.runners.direct.ParDoMultiOverrideFactory.DIRECT_STATEFUL_PAR_DO_URN; +import static org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory.DIRECT_TEST_STREAM_URN; +import static org.apache.beam.runners.direct.ViewOverrideFactory.DIRECT_WRITE_VIEW_URN; +import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableMap; import java.util.ArrayList; import java.util.Collection; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems; -import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; -import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; -import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; -import org.apache.beam.runners.direct.ViewOverrideFactory.WriteView; -import org.apache.beam.sdk.io.Read; +import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems.ProcessElements; +import org.apache.beam.runners.core.construction.PTransformTranslation; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.runners.core.construction.SdkComponents; +import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory.DirectTestStream; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.transforms.Flatten.PCollections; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.windowing.Window; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,43 +55,93 @@ */ class TransformEvaluatorRegistry implements TransformEvaluatorFactory { private static final Logger LOG = LoggerFactory.getLogger(TransformEvaluatorRegistry.class); + public static TransformEvaluatorRegistry defaultRegistry(EvaluationContext ctxt) { - @SuppressWarnings({"rawtypes"}) - ImmutableMap, TransformEvaluatorFactory> primitives = - ImmutableMap., TransformEvaluatorFactory>builder() - .put(Read.Bounded.class, new BoundedReadEvaluatorFactory(ctxt)) - .put(Read.Unbounded.class, new UnboundedReadEvaluatorFactory(ctxt)) + ImmutableMap primitives = + ImmutableMap.builder() + // Beam primitives + .put(READ_TRANSFORM_URN, new ReadEvaluatorFactory(ctxt)) .put( - ParDo.MultiOutput.class, + PAR_DO_TRANSFORM_URN, new ParDoEvaluatorFactory<>(ctxt, ParDoEvaluator.defaultRunnerFactory())) - .put(StatefulParDo.class, new StatefulParDoEvaluatorFactory<>(ctxt)) - .put(PCollections.class, new FlattenEvaluatorFactory(ctxt)) - .put(WriteView.class, new ViewEvaluatorFactory(ctxt)) - .put(Window.Assign.class, new WindowEvaluatorFactory(ctxt)) - // Runner-specific primitives used in expansion of GroupByKey - .put(DirectGroupByKeyOnly.class, new GroupByKeyOnlyEvaluatorFactory(ctxt)) - .put(DirectGroupAlsoByWindow.class, new GroupAlsoByWindowEvaluatorFactory(ctxt)) - .put( - TestStreamEvaluatorFactory.DirectTestStreamFactory.DirectTestStream.class, - new TestStreamEvaluatorFactory(ctxt)) - // Runner-specific primitive used in expansion of SplittableParDo - .put( - SplittableParDoViaKeyedWorkItems.ProcessElements.class, - new SplittableProcessElementsEvaluatorFactory<>(ctxt)) + .put(FLATTEN_TRANSFORM_URN, new FlattenEvaluatorFactory(ctxt)) + .put(WINDOW_TRANSFORM_URN, new WindowEvaluatorFactory(ctxt)) + + // Runner-specific primitives + .put(DIRECT_WRITE_VIEW_URN, new ViewEvaluatorFactory(ctxt)) + .put(DIRECT_STATEFUL_PAR_DO_URN, new StatefulParDoEvaluatorFactory<>(ctxt)) + .put(DIRECT_GBKO_URN, new GroupByKeyOnlyEvaluatorFactory(ctxt)) + .put(DIRECT_GABW_URN, new GroupAlsoByWindowEvaluatorFactory(ctxt)) + .put(DIRECT_TEST_STREAM_URN, new TestStreamEvaluatorFactory(ctxt)) + + // Runners-core primitives + .put(SPLITTABLE_PROCESS_URN, new SplittableProcessElementsEvaluatorFactory<>(ctxt)) .build(); return new TransformEvaluatorRegistry(primitives); } + /** Registers classes specialized to the direct runner. */ + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class DirectTransformsRegistrar implements TransformPayloadTranslatorRegistrar { + @Override + public Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> + getTransformPayloadTranslators() { + return ImmutableMap + ., PTransformTranslation.TransformPayloadTranslator>builder() + .put( + DirectGroupByKey.DirectGroupByKeyOnly.class, + new PTransformTranslation.RawPTransformTranslator<>()) + .put( + DirectGroupByKey.DirectGroupAlsoByWindow.class, + new PTransformTranslation.RawPTransformTranslator()) + .put( + ParDoMultiOverrideFactory.StatefulParDo.class, + new PTransformTranslation.RawPTransformTranslator<>()) + .put( + ViewOverrideFactory.WriteView.class, + new PTransformTranslation.RawPTransformTranslator<>()) + .put(DirectTestStream.class, new PTransformTranslation.RawPTransformTranslator<>()) + .put( + SplittableParDoViaKeyedWorkItems.ProcessElements.class, + new SplittableParDoProcessElementsTranslator()) + .build(); + } + } + + /** + * A translator just to vend the URN. This will need to be moved to runners-core-construction-java + * once SDF is reorganized appropriately. + */ + private static class SplittableParDoProcessElementsTranslator + implements TransformPayloadTranslator> { + + private SplittableParDoProcessElementsTranslator() {} + + @Override + public String getUrn(ProcessElements transform) { + return SPLITTABLE_PROCESS_URN; + } + + @Override + public FunctionSpec translate( + AppliedPTransform> transform, SdkComponents components) { + throw new UnsupportedOperationException( + String.format("%s should never be translated", + ProcessElements.class.getCanonicalName())); + } + } + // the TransformEvaluatorFactories can construct instances of all generic types of transform, // so all instances of a primitive can be handled with the same evaluator factory. - @SuppressWarnings("rawtypes") - private final Map, TransformEvaluatorFactory> factories; + private final Map factories; private final AtomicBoolean finished = new AtomicBoolean(false); private TransformEvaluatorRegistry( @SuppressWarnings("rawtypes") - Map, TransformEvaluatorFactory> factories) { + Map factories) { this.factories = factories; } @@ -91,10 +151,12 @@ public TransformEvaluator forApplication( throws Exception { checkState( !finished.get(), "Tried to get an evaluator for a finished TransformEvaluatorRegistry"); - Class transformClass = application.getTransform().getClass(); + + String urn = PTransformTranslation.urnForTransform(application.getTransform()); + TransformEvaluatorFactory factory = checkNotNull( - factories.get(transformClass), "No evaluator for PTransform type %s", transformClass); + factories.get(urn), "No evaluator for PTransform \"%s\"", urn); return factory.forApplication(application, inputBundle); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java index cba826ccc98c..7d4bba112385 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; +import org.apache.beam.runners.core.construction.ReadTranslation; import org.apache.beam.runners.direct.UnboundedReadDeduplicator.NeverDeduplicator; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.Read.Unbounded; @@ -253,7 +254,8 @@ public TransformResult> finishBun } @AutoValue - abstract static class UnboundedSourceShard { + abstract static class UnboundedSourceShard + implements SourceShard { static UnboundedSourceShard unstarted( UnboundedSource source, UnboundedReadDeduplicator deduplicator) { return of(source, deduplicator, null, null); @@ -268,7 +270,8 @@ static UnboundedSourceShard getSource(); + @Override + public abstract UnboundedSource getSource(); abstract UnboundedReadDeduplicator getDeduplicator(); @@ -283,9 +286,8 @@ UnboundedSourceShard withCheckpoint(CheckpointT newCheckpoint) { } } - static class InputProvider - implements RootInputProvider< - OutputT, UnboundedSourceShard, PBegin, Unbounded> { + static class InputProvider + implements RootInputProvider, PBegin> { private final EvaluationContext evaluationContext; InputProvider(EvaluationContext evaluationContext) { @@ -293,27 +295,28 @@ static class InputProvider } @Override - public Collection>> getInitialInputs( - AppliedPTransform, Unbounded> transform, + public Collection>> getInitialInputs( + AppliedPTransform, PTransform>> + transform, int targetParallelism) throws Exception { - UnboundedSource source = transform.getTransform().getSource(); - List> splits = + UnboundedSource source = ReadTranslation.unboundedSourceFromTransform(transform); + List> splits = source.split(targetParallelism, evaluationContext.getPipelineOptions()); UnboundedReadDeduplicator deduplicator = source.requiresDeduping() ? UnboundedReadDeduplicator.CachedIdDeduplicator.create() : NeverDeduplicator.create(); - ImmutableList.Builder>> initialShards = + ImmutableList.Builder>> initialShards = ImmutableList.builder(); - for (UnboundedSource split : splits) { - UnboundedSourceShard shard = + for (UnboundedSource split : splits) { + UnboundedSourceShard shard = UnboundedSourceShard.unstarted(split, deduplicator); initialShards.add( evaluationContext - .>createRootBundle() - .add(WindowedValue.>valueInGlobalWindow(shard)) + .>createRootBundle() + .add(WindowedValue.>valueInGlobalWindow(shard)) .commit(BoundedWindow.TIMESTAMP_MAX_VALUE)); } return initialShards.build(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewOverrideFactory.java index b3bbac827fd3..501b4365835a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewOverrideFactory.java @@ -18,10 +18,12 @@ package org.apache.beam.runners.direct; +import com.google.protobuf.Message; import java.util.Collections; import java.util.Map; import org.apache.beam.runners.core.construction.ForwardingPTransform; import org.apache.beam.runners.core.construction.PTransformReplacements; +import org.apache.beam.runners.core.construction.PTransformTranslation.RawPTransform; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -93,7 +95,7 @@ protected PTransform, PCollectionView> delegate() { * to {@link ViewT}. */ static final class WriteView - extends PTransform>, PCollectionView> { + extends RawPTransform>, PCollectionView, Message> { private final CreatePCollectionView og; WriteView(CreatePCollectionView og) { @@ -110,5 +112,13 @@ public PCollectionView expand(PCollection> input) { public PCollectionView getView() { return og.getView(); } + + @Override + public String getUrn() { + return DIRECT_WRITE_VIEW_URN; + } } + + public static final String DIRECT_WRITE_VIEW_URN = + "urn:beam:directrunner:transforms:write_view:v1"; } diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml index 8ff0cb02eb3c..3430750d37a8 100644 --- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml +++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml @@ -405,4 +405,11 @@ + + + + + + + From 6b21ea660eb82c486d17c5d8041c90c6479a54dc Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 25 May 2017 10:30:44 -0700 Subject: [PATCH 089/159] More robust gen_protos on jenkins. Don't use the globally shared build directory, which pip may refuse to use if there's a failed install. --- sdks/python/gen_protos.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/sdks/python/gen_protos.py b/sdks/python/gen_protos.py index edaaff488dfb..bb8592d985c3 100644 --- a/sdks/python/gen_protos.py +++ b/sdks/python/gen_protos.py @@ -22,6 +22,7 @@ import multiprocessing import os import pkg_resources +import shutil import subprocess import sys import warnings @@ -55,10 +56,11 @@ def generate_proto_files(): if out_files and not proto_files: # We have out_files but no protos; assume they're up to date. # This is actually the common case (e.g. installation from an sdist). + logging.info('No proto files; using existing generated files.') return elif not out_files and not proto_files: - if not common: + if not os.path.exists(common): raise RuntimeError( 'Not in apache git tree; unable to find proto definitions.') else: @@ -105,9 +107,13 @@ def generate_proto_files(): def _install_grpcio_tools_and_generate_proto_files(): install_path = os.path.join( os.path.dirname(os.path.abspath(__file__)), '.eggs', 'grpcio-wheels') - logging.warning('Downloading a grpcio-tools to %s' % install_path) + build_path = install_path + '-build' + if os.path.exists(build_path): + shutil.rmtree(build_path) + logging.warning('Installing grpcio-tools into %s' % install_path) subprocess.check_call( - ['pip', 'install', '-t', install_path, '--upgrade', GRPC_TOOLS]) + ['pip', 'install', '--target', install_path, '--build', build_path, + '--upgrade', GRPC_TOOLS]) sys.path.append(install_path) generate_proto_files() From 73fbcba88fcc746695a0174d2e69bec4580c1d23 Mon Sep 17 00:00:00 2001 From: Ben Chambers Date: Tue, 23 May 2017 15:44:48 -0700 Subject: [PATCH 090/159] Clarify that PTransform#expand shouldn't be called --- .../java/org/apache/beam/sdk/transforms/PTransform.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java index d5df944330b6..58051df78215 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java @@ -176,8 +176,12 @@ public abstract class PTransform implements Serializable /* See the note above */, HasDisplayData { /** - * Applies this {@code PTransform} on the given {@code InputT}, and returns its - * {@code Output}. + * Override this method to specify how this {@code PTransform} should be expanded + * on the given {@code InputT}. + * + *

NOTE: This method should not be called directly. Instead apply the + * {@code PTransform} should be applied to the {@code InputT} using the {@code apply} + * method. * *

Composite transforms, which are defined in terms of other transforms, * should return the output of one of the composed transforms. Non-composite From 73305d631255d173a0383673700c211055fc818e Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 25 May 2017 13:43:19 -0700 Subject: [PATCH 091/159] Fixup CombineTranslation --- .../beam/runners/core/construction/CombineTranslation.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java index e0b6d5c9abf8..855fba740a02 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java @@ -105,7 +105,7 @@ private static SdkFunctionSpec toProto(GlobalCombineFn combineFn) { public static Coder getAccumulatorCoder( CombinePayload payload, RunnerApi.Components components) throws IOException { String id = payload.getAccumulatorCoderId(); - return Coders.fromProto(components.getCodersOrThrow(id), components); + return CoderTranslation.fromProto(components.getCodersOrThrow(id), components); } public static GlobalCombineFn getCombineFn(CombinePayload payload) From 281eaab3a0ac3810733b87159c9ea9e82a8480fd Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 19 May 2017 16:09:13 -0700 Subject: [PATCH 092/159] Cleanups in SimpleDoFnRunner and ParDoEvaluator - Makes it an error to output to an undeclared output tag, instead of effectively silently dropping this data. - Removes code in SimpleDoFnRunner that, IIRC, assigned windows to outputs from bundle methods - which is now obsolete because you have to specify the window when outputting from FinishBundle, and you can't output from StartBundle at all. --- .../beam/runners/core/SimpleDoFnRunner.java | 387 ++++-------------- .../beam/runners/direct/ParDoEvaluator.java | 18 +- .../spark/translation/MultiDoFnFunction.java | 8 +- .../translation/TransformTranslator.java | 1 + .../StreamingTransformTranslator.java | 1 + .../apache/beam/sdk/transforms/ParDoTest.java | 76 +--- 6 files changed, 102 insertions(+), 389 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 97b0b3307740..7d7babd1397c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -20,11 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.FluentIterable; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.Set; import javax.annotation.Nullable; @@ -37,20 +36,13 @@ import org.apache.beam.sdk.state.Timer; import org.apache.beam.sdk.state.TimerSpec; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.FinishBundleContext; -import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; -import org.apache.beam.sdk.transforms.DoFn.ProcessContext; -import org.apache.beam.sdk.transforms.DoFn.StartBundleContext; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; @@ -73,18 +65,19 @@ */ public class SimpleDoFnRunner implements DoFnRunner { + private final PipelineOptions options; /** The {@link DoFn} being run. */ private final DoFn fn; /** The {@link DoFnInvoker} being run. */ private final DoFnInvoker invoker; - /** The context used for running the {@link DoFn}. */ - private final DoFnContext context; - + private final SideInputReader sideInputReader; private final OutputManager outputManager; private final TupleTag mainOutputTag; + /** The set of known output tags. */ + private final Set> outputTags; private final boolean observesWindow; @@ -106,12 +99,16 @@ public SimpleDoFnRunner( List> additionalOutputTags, StepContext stepContext, WindowingStrategy windowingStrategy) { + this.options = options; this.fn = fn; this.signature = DoFnSignatures.getSignature(fn.getClass()); this.observesWindow = signature.processElement().observesWindow() || !sideInputReader.isEmpty(); this.invoker = DoFnInvokers.invokerFor(fn); + this.sideInputReader = sideInputReader; this.outputManager = outputManager; this.mainOutputTag = mainOutputTag; + this.outputTags = + Sets.newHashSet(FluentIterable.>of(mainOutputTag).append(additionalOutputTags)); this.stepContext = stepContext; // This is a cast of an _invariant_ coder. But we are assured by pipeline validation @@ -121,26 +118,13 @@ public SimpleDoFnRunner( (Coder) windowingStrategy.getWindowFn().windowCoder(); this.windowCoder = untypedCoder; this.allowedLateness = windowingStrategy.getAllowedLateness(); - - this.context = - new DoFnContext<>( - options, - fn, - sideInputReader, - outputManager, - mainOutputTag, - additionalOutputTags, - stepContext, - windowingStrategy.getWindowFn()); } @Override public void startBundle() { - DoFnStartBundleContext startBundleContext = - createStartBundleContext(fn, context); // This can contain user code. Wrap it in case it throws an exception. try { - invoker.invokeStartBundle(startBundleContext); + invoker.invokeStartBundle(new DoFnStartBundleContext()); } catch (Throwable t) { // Exception in user code. throw wrapUserCodeException(t); @@ -174,7 +158,7 @@ public void onTimer( case PROCESSING_TIME: case SYNCHRONIZED_PROCESSING_TIME: - effectiveTimestamp = context.stepContext.timerInternals().currentInputWatermarkTime(); + effectiveTimestamp = stepContext.timerInternals().currentInputWatermarkTime(); break; default: @@ -182,18 +166,15 @@ public void onTimer( String.format("Unknown time domain: %s", timeDomain)); } - OnTimerArgumentProvider argumentProvider = - new OnTimerArgumentProvider<>( - fn, context, window, allowedLateness, effectiveTimestamp, timeDomain); + OnTimerArgumentProvider argumentProvider = + new OnTimerArgumentProvider(window, effectiveTimestamp, timeDomain); invoker.invokeOnTimer(timerId, argumentProvider); } private void invokeProcessElement(WindowedValue elem) { - final DoFnProcessContext processContext = createProcessContext(elem); - // This can contain user code. Wrap it in case it throws an exception. try { - invoker.invokeProcessElement(processContext); + invoker.invokeProcessElement(new DoFnProcessContext(elem)); } catch (Exception ex) { throw wrapUserCodeException(ex); } @@ -201,32 +182,15 @@ private void invokeProcessElement(WindowedValue elem) { @Override public void finishBundle() { - DoFnFinishBundleContext finishBundleContext = - createFinishBundleContext(fn, context); // This can contain user code. Wrap it in case it throws an exception. try { - invoker.invokeFinishBundle(finishBundleContext); + invoker.invokeFinishBundle(new DoFnFinishBundleContext()); } catch (Throwable t) { // Exception in user code. throw wrapUserCodeException(t); } } - private DoFnStartBundleContext createStartBundleContext( - DoFn fn, DoFnContext context) { - return new DoFnStartBundleContext<>(fn, context); - } - - private DoFnFinishBundleContext createFinishBundleContext( - DoFn fn, DoFnContext context) { - return new DoFnFinishBundleContext<>(fn, context); - } - - /** Returns a new {@link DoFn.ProcessContext} for the given element. */ - private DoFnProcessContext createProcessContext(WindowedValue elem) { - return new DoFnProcessContext(fn, context, elem, allowedLateness); - } - private RuntimeException wrapUserCodeException(Throwable t) { throw UserCodeException.wrapIf(!isSystemDoFn(), t); } @@ -235,170 +199,31 @@ private boolean isSystemDoFn() { return invoker.getClass().isAnnotationPresent(SystemDoFnInternal.class); } - /** - * A concrete implementation of {@code DoFn.Context} used for running a {@link DoFn}. - * - * @param the type of the {@link DoFn} (main) input elements - * @param the type of the {@link DoFn} (main) output elements - */ - private static class DoFnContext { - private static final int MAX_SIDE_OUTPUTS = 1000; - - final PipelineOptions options; - final DoFn fn; - final SideInputReader sideInputReader; - final OutputManager outputManager; - final TupleTag mainOutputTag; - final StepContext stepContext; - final WindowFn windowFn; - - /** - * The set of known output tags, some of which may be undeclared, so we can throw an exception - * when it exceeds {@link #MAX_SIDE_OUTPUTS}. - */ - private Set> outputTags; - - public DoFnContext( - PipelineOptions options, - DoFn fn, - SideInputReader sideInputReader, - OutputManager outputManager, - TupleTag mainOutputTag, - List> additionalOutputTags, - StepContext stepContext, - WindowFn windowFn) { - this.options = options; - this.fn = fn; - this.sideInputReader = sideInputReader; - this.outputManager = outputManager; - this.mainOutputTag = mainOutputTag; - this.outputTags = Sets.newHashSet(); - - outputTags.add(mainOutputTag); - for (TupleTag additionalOutputTag : additionalOutputTags) { - outputTags.add(additionalOutputTag); - } - - this.stepContext = stepContext; - this.windowFn = windowFn; - } - - ////////////////////////////////////////////////////////////////////////////// - - public PipelineOptions getPipelineOptions() { - return options; - } - - WindowedValue makeWindowedValue( - T output, Instant timestamp, Collection windows, PaneInfo pane) { - final Instant inputTimestamp = timestamp; - - if (timestamp == null) { - timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; - } - - if (windows == null) { - try { - // The windowFn can never succeed at accessing the element, so its type does not - // matter here - @SuppressWarnings("unchecked") - WindowFn objectWindowFn = (WindowFn) windowFn; - windows = - objectWindowFn.assignWindows( - objectWindowFn.new AssignContext() { - @Override - public Object element() { - throw new UnsupportedOperationException( - "WindowFn attempted to access input element when none was available"); - } - - @Override - public Instant timestamp() { - if (inputTimestamp == null) { - throw new UnsupportedOperationException( - "WindowFn attempted to access input timestamp when none was available"); - } - return inputTimestamp; - } - - @Override - public W window() { - throw new UnsupportedOperationException( - "WindowFn attempted to access input windows when none were available"); - } - }); - } catch (Exception e) { - throw UserCodeException.wrap(e); - } - } - - return WindowedValue.of(output, timestamp, windows, pane); - } - - public T sideInput(PCollectionView view, BoundedWindow sideInputWindow) { - if (!sideInputReader.contains(view)) { - throw new IllegalArgumentException("calling sideInput() with unknown view"); - } - return sideInputReader.get(view, sideInputWindow); - } - - void outputWindowedValue( - OutputT output, - Instant timestamp, - Collection windows, - PaneInfo pane) { - outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane)); - } - - void outputWindowedValue(WindowedValue windowedElem) { - outputManager.output(mainOutputTag, windowedElem); - } - - private void outputWindowedValue( - TupleTag tag, - T output, - Instant timestamp, - Collection windows, - PaneInfo pane) { - outputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane)); - } - - private void outputWindowedValue(TupleTag tag, WindowedValue windowedElem) { - if (!outputTags.contains(tag)) { - // This tag wasn't declared nor was it seen before during this execution. - // Thus, this must be a new, undeclared and unconsumed output. - // To prevent likely user errors, enforce the limit on the number of side - // outputs. - if (outputTags.size() >= MAX_SIDE_OUTPUTS) { - throw new IllegalArgumentException( - "the number of outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS); - } - outputTags.add(tag); - } - - outputManager.output(tag, windowedElem); + private T sideInput(PCollectionView view, BoundedWindow sideInputWindow) { + if (!sideInputReader.contains(view)) { + throw new IllegalArgumentException("calling sideInput() with unknown view"); } + return sideInputReader.get(view, sideInputWindow); } + private void outputWindowedValue(TupleTag tag, WindowedValue windowedElem) { + checkArgument(outputTags.contains(tag), "Unknown output tag %s", tag); + outputManager.output(tag, windowedElem); + } /** * A concrete implementation of {@link DoFn.StartBundleContext}. */ - private class DoFnStartBundleContext + private class DoFnStartBundleContext extends DoFn.StartBundleContext implements DoFnInvoker.ArgumentProvider { - private final DoFn fn; - private final DoFnContext context; - - private DoFnStartBundleContext(DoFn fn, DoFnContext context) { + private DoFnStartBundleContext() { fn.super(); - this.fn = fn; - this.context = context; } @Override public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); + return options; } @Override @@ -408,24 +233,25 @@ public BoundedWindow window() { } @Override - public StartBundleContext startBundleContext(DoFn doFn) { + public DoFn.StartBundleContext startBundleContext(DoFn doFn) { return this; } @Override - public FinishBundleContext finishBundleContext(DoFn doFn) { + public DoFn.FinishBundleContext finishBundleContext( + DoFn doFn) { throw new UnsupportedOperationException( "Cannot access FinishBundleContext outside of @FinishBundle method."); } @Override - public ProcessContext processContext(DoFn doFn) { + public DoFn.ProcessContext processContext(DoFn doFn) { throw new UnsupportedOperationException( "Cannot access ProcessContext outside of @ProcessElement method."); } @Override - public OnTimerContext onTimerContext(DoFn doFn) { + public DoFn.OnTimerContext onTimerContext(DoFn doFn) { throw new UnsupportedOperationException( "Cannot access OnTimerContext outside of @OnTimer methods."); } @@ -453,20 +279,16 @@ public Timer timer(String timerId) { * B * A concrete implementation of {@link DoFn.FinishBundleContext}. */ - private class DoFnFinishBundleContext + private class DoFnFinishBundleContext extends DoFn.FinishBundleContext implements DoFnInvoker.ArgumentProvider { - private final DoFnContext context; - - private DoFnFinishBundleContext( - DoFn fn, DoFnContext context) { + private DoFnFinishBundleContext() { fn.super(); - this.context = context; } @Override public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); + return options; } @Override @@ -476,24 +298,25 @@ public BoundedWindow window() { } @Override - public StartBundleContext startBundleContext(DoFn doFn) { + public DoFn.StartBundleContext startBundleContext(DoFn doFn) { throw new UnsupportedOperationException( "Cannot access StartBundleContext outside of @StartBundle method."); } @Override - public FinishBundleContext finishBundleContext(DoFn doFn) { + public DoFn.FinishBundleContext finishBundleContext( + DoFn doFn) { return this; } @Override - public ProcessContext processContext(DoFn doFn) { + public DoFn.ProcessContext processContext(DoFn doFn) { throw new UnsupportedOperationException( "Cannot access ProcessContext outside of @ProcessElement method."); } @Override - public OnTimerContext onTimerContext(DoFn doFn) { + public DoFn.OnTimerContext onTimerContext(DoFn doFn) { throw new UnsupportedOperationException( "Cannot access OnTimerContext outside of @OnTimer methods."); } @@ -518,30 +341,22 @@ public Timer timer(String timerId) { @Override public void output(OutputT output, Instant timestamp, BoundedWindow window) { - context.outputWindowedValue(WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); + output(mainOutputTag, output, timestamp, window); } @Override public void output(TupleTag tag, T output, Instant timestamp, BoundedWindow window) { - context.outputWindowedValue( - tag, WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); + outputWindowedValue(tag, WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); } } /** * A concrete implementation of {@link DoFn.ProcessContext} used for running a {@link DoFn} over a * single element. - * - * @param the type of the {@link DoFn} (main) input elements - * @param the type of the {@link DoFn} (main) output elements */ - private class DoFnProcessContext extends DoFn.ProcessContext + private class DoFnProcessContext extends DoFn.ProcessContext implements DoFnInvoker.ArgumentProvider { - - final DoFn fn; - final DoFnContext context; - final WindowedValue windowedValue; - private final Duration allowedLateness; + final WindowedValue elem; /** Lazily initialized; should only be accessed via {@link #getNamespace()}. */ @Nullable private StateNamespace namespace; @@ -549,7 +364,7 @@ private class DoFnProcessContext extends DoFn. /** * The state namespace for this context. * - *

Any call to {@link #getNamespace()} when more than one window is present will crash; this + *

Any call to this method when more than one window is present will crash; this * represents a bug in the runner or the {@link DoFnSignature}, since values must be in exactly * one window when state or timers are relevant. */ @@ -561,55 +376,32 @@ private StateNamespace getNamespace() { } private DoFnProcessContext( - DoFn fn, - DoFnContext context, - WindowedValue windowedValue, - Duration allowedLateness) { + WindowedValue elem) { fn.super(); - this.fn = fn; - this.context = context; - this.windowedValue = windowedValue; - this.allowedLateness = allowedLateness; + this.elem = elem; } @Override public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); + return options; } @Override public InputT element() { - return windowedValue.getValue(); + return elem.getValue(); } @Override public T sideInput(PCollectionView view) { checkNotNull(view, "View passed to sideInput cannot be null"); - Iterator windowIter = windows().iterator(); - BoundedWindow window; - if (!windowIter.hasNext()) { - if (context.windowFn instanceof GlobalWindows) { - // TODO: Remove this once GroupByKeyOnly no longer outputs elements - // without windows - window = GlobalWindow.INSTANCE; - } else { - throw new IllegalStateException( - "sideInput called when main input element is not in any windows"); - } - } else { - window = windowIter.next(); - if (windowIter.hasNext()) { - throw new IllegalStateException( - "sideInput called when main input element is in multiple windows"); - } - } - return context.sideInput( + BoundedWindow window = Iterables.getOnlyElement(windows()); + return SimpleDoFnRunner.this.sideInput( view, view.getWindowMappingFn().getSideInputWindow(window)); } @Override public PaneInfo pane() { - return windowedValue.getPane(); + return elem.getPane(); } @Override @@ -619,37 +411,36 @@ public void updateWatermark(Instant watermark) { @Override public void output(OutputT output) { - context.outputWindowedValue(windowedValue.withValue(output)); + output(mainOutputTag, output); } @Override public void outputWithTimestamp(OutputT output, Instant timestamp) { checkTimestamp(timestamp); - context.outputWindowedValue( - output, timestamp, windowedValue.getWindows(), windowedValue.getPane()); + outputWithTimestamp(mainOutputTag, output, timestamp); } @Override public void output(TupleTag tag, T output) { checkNotNull(tag, "Tag passed to output cannot be null"); - context.outputWindowedValue(tag, windowedValue.withValue(output)); + outputWindowedValue(tag, elem.withValue(output)); } @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { checkNotNull(tag, "Tag passed to outputWithTimestamp cannot be null"); checkTimestamp(timestamp); - context.outputWindowedValue( - tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane()); + outputWindowedValue( + tag, WindowedValue.of(output, timestamp, elem.getWindows(), elem.getPane())); } @Override public Instant timestamp() { - return windowedValue.getTimestamp(); + return elem.getTimestamp(); } public Collection windows() { - return windowedValue.getWindows(); + return elem.getWindows(); } @SuppressWarnings("deprecation") // Allowed Skew is deprecated for users, but must be respected @@ -657,7 +448,7 @@ private void checkTimestamp(Instant timestamp) { // The documentation of getAllowedTimestampSkew explicitly permits Long.MAX_VALUE to be used // for infinite skew. Defend against underflow in that case for timestamps before the epoch if (fn.getAllowedTimestampSkew().getMillis() != Long.MAX_VALUE - && timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) { + && timestamp.isBefore(elem.getTimestamp().minus(fn.getAllowedTimestampSkew()))) { throw new IllegalArgumentException( String.format( "Cannot output with timestamp %s. Output timestamps must be no earlier than the " @@ -665,23 +456,24 @@ private void checkTimestamp(Instant timestamp) { + "DoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed " + "skew.", timestamp, - windowedValue.getTimestamp(), + elem.getTimestamp(), PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod()))); } } @Override public BoundedWindow window() { - return Iterables.getOnlyElement(windowedValue.getWindows()); + return Iterables.getOnlyElement(elem.getWindows()); } @Override - public StartBundleContext startBundleContext(DoFn doFn) { + public DoFn.StartBundleContext startBundleContext(DoFn doFn) { throw new UnsupportedOperationException("StartBundleContext parameters are not supported."); } @Override - public FinishBundleContext finishBundleContext(DoFn doFn) { + public DoFn.FinishBundleContext finishBundleContext( + DoFn doFn) { throw new UnsupportedOperationException("FinishBundleContext parameters are not supported."); } @@ -691,7 +483,7 @@ public DoFn.ProcessContext processContext(DoFn } @Override - public OnTimerContext onTimerContext(DoFn doFn) { + public DoFn.OnTimerContext onTimerContext(DoFn doFn) { throw new UnsupportedOperationException( "Cannot access OnTimerContext outside of @OnTimer methods."); } @@ -720,7 +512,7 @@ public Timer timer(String timerId) { TimerSpec spec = (TimerSpec) signature.timerDeclarations().get(timerId).field().get(fn); return new TimerInternalsTimer( - window(), getNamespace(), allowedLateness, timerId, spec, stepContext.timerInternals()); + window(), getNamespace(), timerId, spec, stepContext.timerInternals()); } catch (IllegalAccessException e) { throw new RuntimeException(e); } @@ -730,20 +522,13 @@ public Timer timer(String timerId) { /** * A concrete implementation of {@link DoFnInvoker.ArgumentProvider} used for running a {@link * DoFn} on a timer. - * - * @param the type of the {@link DoFn} (main) input elements - * @param the type of the {@link DoFn} (main) output elements */ - private class OnTimerArgumentProvider + private class OnTimerArgumentProvider extends DoFn.OnTimerContext implements DoFnInvoker.ArgumentProvider { - - final DoFn fn; - final DoFnContext context; private final BoundedWindow window; private final Instant timestamp; private final TimeDomain timeDomain; - private final Duration allowedLateness; /** Lazily initialized; should only be accessed via {@link #getNamespace()}. */ private StateNamespace namespace; @@ -751,7 +536,7 @@ private class OnTimerArgumentProvider /** * The state namespace for this context. * - *

Any call to {@link #getNamespace()} when more than one window is present will crash; this + *

Any call to this method when more than one window is present will crash; this * represents a bug in the runner or the {@link DoFnSignature}, since values must be in exactly * one window when state or timers are relevant. */ @@ -763,17 +548,11 @@ private StateNamespace getNamespace() { } private OnTimerArgumentProvider( - DoFn fn, - DoFnContext context, BoundedWindow window, - Duration allowedLateness, Instant timestamp, TimeDomain timeDomain) { fn.super(); - this.fn = fn; - this.context = context; this.window = window; - this.allowedLateness = allowedLateness; this.timestamp = timestamp; this.timeDomain = timeDomain; } @@ -789,12 +568,13 @@ public BoundedWindow window() { } @Override - public StartBundleContext startBundleContext(DoFn doFn) { + public DoFn.StartBundleContext startBundleContext(DoFn doFn) { throw new UnsupportedOperationException("StartBundleContext parameters are not supported."); } @Override - public FinishBundleContext finishBundleContext(DoFn doFn) { + public DoFn.FinishBundleContext finishBundleContext( + DoFn doFn) { throw new UnsupportedOperationException("FinishBundleContext parameters are not supported."); } @@ -805,12 +585,12 @@ public TimeDomain timeDomain() { @Override - public ProcessContext processContext(DoFn doFn) { + public DoFn.ProcessContext processContext(DoFn doFn) { throw new UnsupportedOperationException("ProcessContext parameters are not supported."); } @Override - public OnTimerContext onTimerContext(DoFn doFn) { + public DoFn.OnTimerContext onTimerContext(DoFn doFn) { return this; } @@ -838,7 +618,7 @@ public Timer timer(String timerId) { TimerSpec spec = (TimerSpec) signature.timerDeclarations().get(timerId).field().get(fn); return new TimerInternalsTimer( - window, getNamespace(), allowedLateness, timerId, spec, stepContext.timerInternals()); + window, getNamespace(), timerId, spec, stepContext.timerInternals()); } catch (IllegalAccessException e) { throw new RuntimeException(e); } @@ -846,42 +626,37 @@ public Timer timer(String timerId) { @Override public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); + return options; } @Override public void output(OutputT output) { - context.outputWindowedValue( - output, timestamp(), Collections.singleton(window()), PaneInfo.NO_FIRING); + output(mainOutputTag, output); } @Override public void outputWithTimestamp(OutputT output, Instant timestamp) { - context.outputWindowedValue( - output, timestamp, Collections.singleton(window()), PaneInfo.NO_FIRING); + outputWithTimestamp(mainOutputTag, output, timestamp); } @Override public void output(TupleTag tag, T output) { - context.outputWindowedValue( - tag, output, timestamp, Collections.singleton(window()), PaneInfo.NO_FIRING); + outputWindowedValue(tag, WindowedValue.of(output, timestamp, window(), PaneInfo.NO_FIRING)); } @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - context.outputWindowedValue( - tag, output, timestamp, Collections.singleton(window()), PaneInfo.NO_FIRING); + outputWindowedValue(tag, WindowedValue.of(output, timestamp, window(), PaneInfo.NO_FIRING)); } } - private static class TimerInternalsTimer implements Timer { + private class TimerInternalsTimer implements Timer { private final TimerInternals timerInternals; // The window and the namespace represent the same thing, but the namespace is a cached // and specially encoded form. Since the namespace can be cached across timers, it is // passed in whole rather than being computed here. private final BoundedWindow window; - private final Duration allowedLateness; private final StateNamespace namespace; private final String timerId; private final TimerSpec spec; @@ -891,12 +666,10 @@ private static class TimerInternalsTimer implements Timer { public TimerInternalsTimer( BoundedWindow window, StateNamespace namespace, - Duration allowedLateness, String timerId, TimerSpec spec, TimerInternals timerInternals) { this.window = window; - this.allowedLateness = allowedLateness; this.namespace = namespace; this.timerId = timerId; this.spec = spec; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 28fc68d293cb..26da6c656ade 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -17,8 +17,9 @@ */ package org.apache.beam.runners.direct; +import static com.google.common.base.Preconditions.checkArgument; + import com.google.common.collect.ImmutableList; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -218,7 +219,6 @@ public TransformResult finishBundle() { static class BundleOutputManager implements OutputManager { private final Map, UncommittedBundle> bundles; - private final Map, List> undeclaredOutputs; public static BundleOutputManager create(Map, UncommittedBundle> outputBundles) { return new BundleOutputManager(outputBundles); @@ -226,23 +226,13 @@ public static BundleOutputManager create(Map, UncommittedBundle> private BundleOutputManager(Map, UncommittedBundle> bundles) { this.bundles = bundles; - undeclaredOutputs = new HashMap<>(); } @SuppressWarnings({"unchecked", "rawtypes"}) @Override public void output(TupleTag tag, WindowedValue output) { - UncommittedBundle bundle = bundles.get(tag); - if (bundle == null) { - List> undeclaredContents = (List) undeclaredOutputs.get(tag); - if (undeclaredContents == null) { - undeclaredContents = new ArrayList<>(); - undeclaredOutputs.put(tag, undeclaredContents); - } - undeclaredContents.add(output); - } else { - bundle.add(output); - } + checkArgument(bundles.containsKey(tag), "Unknown output tag %s", tag); + ((UncommittedBundle) bundles.get(tag)).add(output); } } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java index 4a665417a078..32749122d5b2 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java @@ -22,8 +22,8 @@ import com.google.common.collect.Iterators; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.Multimap; -import java.util.Collections; import java.util.Iterator; +import java.util.List; import java.util.Map; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; @@ -57,6 +57,7 @@ public class MultiDoFnFunction private final DoFn doFn; private final SparkRuntimeContext runtimeContext; private final TupleTag mainOutputTag; + private final List> additionalOutputTags; private final Map, KV, SideInputBroadcast>> sideInputs; private final WindowingStrategy windowingStrategy; @@ -66,6 +67,7 @@ public class MultiDoFnFunction * @param doFn The {@link DoFn} to be wrapped. * @param runtimeContext The {@link SparkRuntimeContext}. * @param mainOutputTag The main output {@link TupleTag}. + * @param additionalOutputTags Additional {@link TupleTag output tags}. * @param sideInputs Side inputs used in this {@link DoFn}. * @param windowingStrategy Input {@link WindowingStrategy}. */ @@ -76,6 +78,7 @@ public MultiDoFnFunction( DoFn doFn, SparkRuntimeContext runtimeContext, TupleTag mainOutputTag, + List> additionalOutputTags, Map, KV, SideInputBroadcast>> sideInputs, WindowingStrategy windowingStrategy) { this.aggAccum = aggAccum; @@ -84,6 +87,7 @@ public MultiDoFnFunction( this.doFn = doFn; this.runtimeContext = runtimeContext; this.mainOutputTag = mainOutputTag; + this.additionalOutputTags = additionalOutputTags; this.sideInputs = sideInputs; this.windowingStrategy = windowingStrategy; } @@ -101,7 +105,7 @@ public Iterable, WindowedValue>> call( new SparkSideInputReader(sideInputs), outputManager, mainOutputTag, - Collections.>emptyList(), + additionalOutputTags, new SparkProcessContext.NoOpStepContext(), windowingStrategy); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index b2ed3a9265d6..742ea83539e2 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -369,6 +369,7 @@ public void evaluate( doFn, context.getRuntimeContext(), transform.getMainOutputTag(), + transform.getAdditionalOutputTags().getAll(), TranslationUtils.getSideInputs(transform.getSideInputs(), context), windowingStrategy)); Map, PValue> outputs = context.getOutputs(transform); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index acb4a02f1e94..43f4b756d186 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -411,6 +411,7 @@ public JavaPairRDD, WindowedValue> call( doFn, runtimeContext, transform.getMainOutputTag(), + transform.getAdditionalOutputTags().getAll(), sideInputs, windowingStrategy)); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index 35c02ba8c9d0..c67cf2a758fe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -32,7 +32,6 @@ import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.not; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; -import static org.hamcrest.collection.IsIterableContainingInOrder.contains; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -515,58 +514,18 @@ public void processElement(ProcessContext c) { @Test @Category(NeedsRunner.class) public void testParDoWritingToUndeclaredTag() { - List inputs = Arrays.asList(3, -42, 666); TupleTag notOutputTag = new TupleTag("additional"){}; - PCollection output = pipeline + pipeline .apply(Create.of(inputs)) .apply(ParDo.of(new TestDoFn( Arrays.>asList(), - Arrays.asList(notOutputTag)))); - - PAssert.that(output) - .satisfies(ParDoTest.HasExpectedOutput.forInput(inputs)); + Arrays.asList(notOutputTag))) + /* No call to .withOutputTags - should cause error */); - pipeline.run(); - } - - @Test - // TODO: The exception thrown is runner-specific, even if the behavior is general - @Category(NeedsRunner.class) - public void testParDoUndeclaredTagLimit() { - - PCollection input = pipeline.apply(Create.of(Arrays.asList(3))); - - // Success for a total of 1000 outputs. - input - .apply("Success1000", ParDo.of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - TupleTag specialOutputTag = new TupleTag(){}; - c.output(specialOutputTag, "special"); - c.output(specialOutputTag, "special"); - c.output(specialOutputTag, "special"); - - for (int i = 0; i < 998; i++) { - c.output(new TupleTag(){}, "tag" + i); - } - }})); - pipeline.run(); - - // Failure for a total of 1001 outputs. - input - .apply("Failure1001", ParDo.of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - for (int i = 0; i < 1000; i++) { - c.output(new TupleTag(){}, "output" + i); - } - }})); - - thrown.expect(RuntimeException.class); - thrown.expectMessage("the number of outputs has exceeded a limit"); + thrown.expectMessage("additional"); pipeline.run(); } @@ -1107,43 +1066,32 @@ static class HasExpectedOutput private final List inputs; private final List sideInputs; private final String additionalOutput; - private final boolean ordered; public static HasExpectedOutput forInput(List inputs) { return new HasExpectedOutput( new ArrayList(inputs), new ArrayList(), - "", - false); + ""); } private HasExpectedOutput(List inputs, List sideInputs, - String additionalOutput, - boolean ordered) { + String additionalOutput) { this.inputs = inputs; this.sideInputs = sideInputs; this.additionalOutput = additionalOutput; - this.ordered = ordered; } public HasExpectedOutput andSideInputs(Integer... sideInputValues) { - List sideInputs = new ArrayList<>(); - for (Integer sideInputValue : sideInputValues) { - sideInputs.add(sideInputValue); - } - return new HasExpectedOutput(inputs, sideInputs, additionalOutput, ordered); + return new HasExpectedOutput( + inputs, Arrays.asList(sideInputValues), additionalOutput); } public HasExpectedOutput fromOutput(TupleTag outputTag) { return fromOutput(outputTag.getId()); } public HasExpectedOutput fromOutput(String outputId) { - return new HasExpectedOutput(inputs, sideInputs, outputId, ordered); - } - - public HasExpectedOutput inOrder() { - return new HasExpectedOutput(inputs, sideInputs, additionalOutput, true); + return new HasExpectedOutput(inputs, sideInputs, outputId); } @Override @@ -1179,11 +1127,7 @@ public Void apply(Iterable outputs) { } String[] expectedProcessedsArray = expectedProcesseds.toArray(new String[expectedProcesseds.size()]); - if (!ordered || expectedProcesseds.isEmpty()) { - assertThat(processeds, containsInAnyOrder(expectedProcessedsArray)); - } else { - assertThat(processeds, contains(expectedProcessedsArray)); - } + assertThat(processeds, containsInAnyOrder(expectedProcessedsArray)); for (String finished : finisheds) { assertEquals(additionalOutputPrefix + "finished", finished); From b4ad7cb01848262fbe82987beec45efd87649894 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Thu, 25 May 2017 13:59:09 -0700 Subject: [PATCH 093/159] Fixes pb2.py path in gitignore --- .gitignore | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index 6e077bd1d440..bd419a78beec 100644 --- a/.gitignore +++ b/.gitignore @@ -25,7 +25,7 @@ sdks/python/**/*.egg sdks/python/LICENSE sdks/python/NOTICE sdks/python/README.md -sdks/python/runners/api/*pb2*.* +sdks/python/apache_beam/runners/api/*pb2*.* # Ignore IntelliJ files. .idea/ From a75f41cc575dba2e5d1c650c0226a91d7ebe125a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 25 May 2017 17:07:08 -0700 Subject: [PATCH 094/159] Additional explicit file cleanup in gen_protos. --- sdks/python/gen_protos.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sdks/python/gen_protos.py b/sdks/python/gen_protos.py index bb8592d985c3..92b8414583ea 100644 --- a/sdks/python/gen_protos.py +++ b/sdks/python/gen_protos.py @@ -111,9 +111,12 @@ def _install_grpcio_tools_and_generate_proto_files(): if os.path.exists(build_path): shutil.rmtree(build_path) logging.warning('Installing grpcio-tools into %s' % install_path) - subprocess.check_call( - ['pip', 'install', '--target', install_path, '--build', build_path, - '--upgrade', GRPC_TOOLS]) + try: + subprocess.check_call( + ['pip', 'install', '--target', install_path, '--build', build_path, + '--upgrade', GRPC_TOOLS]) + finally: + shutil.rmtree(build_path) sys.path.append(install_path) generate_proto_files() From 14329b86e0732acf0190e10d9b04c574baaed2e7 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Wed, 24 May 2017 18:14:59 -0700 Subject: [PATCH 095/159] [BEAM-2354] Add a ReadStringsFromPubSub/WriteStringsToPubSub PTransform --- .../apache_beam/examples/streaming_wordcap.py | 6 +- .../examples/streaming_wordcount.py | 6 +- sdks/python/apache_beam/io/gcp/pubsub.py | 87 ++++++++++++++++--- sdks/python/apache_beam/io/gcp/pubsub_test.py | 57 ++++++++++-- .../runners/dataflow/dataflow_runner.py | 8 +- 5 files changed, 132 insertions(+), 32 deletions(-) diff --git a/sdks/python/apache_beam/examples/streaming_wordcap.py b/sdks/python/apache_beam/examples/streaming_wordcap.py index ce43e1f09849..19f9e5f77ed8 100644 --- a/sdks/python/apache_beam/examples/streaming_wordcap.py +++ b/sdks/python/apache_beam/examples/streaming_wordcap.py @@ -44,8 +44,7 @@ def run(argv=None): with beam.Pipeline(argv=pipeline_args) as p: # Read the text file[pattern] into a PCollection. - lines = p | beam.io.Read( - beam.io.PubSubSource(known_args.input_topic)) + lines = p | beam.io.ReadStringsFromPubSub(known_args.input_topic) # Capitalize the characters in each line. transformed = (lines @@ -53,8 +52,7 @@ def run(argv=None): # Write to PubSub. # pylint: disable=expression-not-assigned - transformed | beam.io.Write( - beam.io.PubSubSink(known_args.output_topic)) + transformed | beam.io.WriteStringsToPubSub(known_args.output_topic) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/streaming_wordcount.py b/sdks/python/apache_beam/examples/streaming_wordcount.py index e9d5dbefa835..ed8b5d08dc62 100644 --- a/sdks/python/apache_beam/examples/streaming_wordcount.py +++ b/sdks/python/apache_beam/examples/streaming_wordcount.py @@ -47,8 +47,7 @@ def run(argv=None): with beam.Pipeline(argv=pipeline_args) as p: # Read the text file[pattern] into a PCollection. - lines = p | 'read' >> beam.io.Read( - beam.io.PubSubSource(known_args.input_topic)) + lines = p | beam.io.ReadStringsFromPubSub(known_args.input_topic) # Capitalize the characters in each line. transformed = (lines @@ -63,8 +62,7 @@ def run(argv=None): # Write to PubSub. # pylint: disable=expression-not-assigned - transformed | 'pubsub_write' >> beam.io.Write( - beam.io.PubSubSink(known_args.output_topic)) + transformed | beam.io.WriteStringsToPubSub(known_args.output_topic) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/io/gcp/pubsub.py b/sdks/python/apache_beam/io/gcp/pubsub.py index 103fce0b8714..7b838d23fea6 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/gcp/pubsub.py @@ -22,15 +22,68 @@ from __future__ import absolute_import -from apache_beam import coders +from apache_beam.io.iobase import Read +from apache_beam.io.iobase import Write from apache_beam.runners.dataflow.native_io import iobase as dataflow_io +from apache_beam.transforms import PTransform +from apache_beam.transforms import ParDo from apache_beam.transforms.display import DisplayDataItem -__all__ = ['PubSubSink', 'PubSubSource'] +__all__ = ['ReadStringsFromPubSub', 'WriteStringsToPubSub'] -class PubSubSource(dataflow_io.NativeSource): - """Source for reading from a given Cloud Pub/Sub topic. + +class ReadStringsFromPubSub(PTransform): + """A ``PTransform`` for reading utf-8 string payloads from Cloud Pub/Sub.""" + + def __init__(self, topic, subscription=None, id_label=None): + """Initializes ``ReadStringsFromPubSub``. + + Attributes: + topic: Cloud Pub/Sub topic in the form "/topics//". + subscription: Optional existing Cloud Pub/Sub subscription to use in the + form "projects//subscriptions/". + id_label: The attribute on incoming Pub/Sub messages to use as a unique + record identifier. When specified, the value of this attribute (which + can be any string that uniquely identifies the record) will be used for + deduplication of messages. If not provided, we cannot guarantee + that no duplicate data will be delivered on the Pub/Sub stream. In this + case, deduplication of the stream will be strictly best effort. + """ + super(ReadStringsFromPubSub, self).__init__() + self._source = _PubSubPayloadSource( + topic, + subscription=subscription, + id_label=id_label) + + def expand(self, pvalue): + pcoll = pvalue.pipeline | Read(self._source) + pcoll.element_type = bytes + pcoll = pcoll | 'decode string' >> ParDo(_decodeUtf8String) + pcoll.element_type = unicode + return pcoll + + +class WriteStringsToPubSub(PTransform): + """A ``PTransform`` for writing utf-8 string payloads to Cloud Pub/Sub.""" + + def __init__(self, topic): + """Initializes ``WriteStringsToPubSub``. + + Attributes: + topic: Cloud Pub/Sub topic in the form "/topics//". + """ + super(WriteStringsToPubSub, self).__init__() + self._sink = _PubSubPayloadSink(topic) + + def expand(self, pcoll): + pcoll = pcoll | 'encode string' >> ParDo(_encodeUtf8String) + pcoll.element_type = bytes + return pcoll | Write(self._sink) + + +class _PubSubPayloadSource(dataflow_io.NativeSource): + """Source for the payload of a message as bytes from a Cloud Pub/Sub topic. Attributes: topic: Cloud Pub/Sub topic in the form "/topics//". @@ -42,15 +95,12 @@ class PubSubSource(dataflow_io.NativeSource): deduplication of messages. If not provided, Dataflow cannot guarantee that no duplicate data will be delivered on the Pub/Sub stream. In this case, deduplication of the stream will be strictly best effort. - coder: The Coder to use for decoding incoming Pub/Sub messages. """ - def __init__(self, topic, subscription=None, id_label=None, - coder=coders.StrUtf8Coder()): + def __init__(self, topic, subscription=None, id_label=None): self.topic = topic self.subscription = subscription self.id_label = id_label - self.coder = coder @property def format(self): @@ -70,15 +120,14 @@ def display_data(self): def reader(self): raise NotImplementedError( - 'PubSubSource is not supported in local execution.') + 'PubSubPayloadSource is not supported in local execution.') -class PubSubSink(dataflow_io.NativeSink): - """Sink for writing to a given Cloud Pub/Sub topic.""" +class _PubSubPayloadSink(dataflow_io.NativeSink): + """Sink for the payload of a message as bytes to a Cloud Pub/Sub topic.""" - def __init__(self, topic, coder=coders.StrUtf8Coder()): + def __init__(self, topic): self.topic = topic - self.coder = coder @property def format(self): @@ -90,4 +139,14 @@ def display_data(self): def writer(self): raise NotImplementedError( - 'PubSubSink is not supported in local execution.') + 'PubSubPayloadSink is not supported in local execution.') + + +def _decodeUtf8String(encoded_value): + """Decodes a string in utf-8 format from bytes""" + return encoded_value.decode('utf-8') + + +def _encodeUtf8String(value): + """Encodes a string in utf-8 format to bytes""" + return value.encode('utf-8') diff --git a/sdks/python/apache_beam/io/gcp/pubsub_test.py b/sdks/python/apache_beam/io/gcp/pubsub_test.py index 1642a95a59e7..322d08a34cb3 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub_test.py +++ b/sdks/python/apache_beam/io/gcp/pubsub_test.py @@ -22,16 +22,53 @@ import hamcrest as hc -from apache_beam.io.gcp.pubsub import PubSubSink -from apache_beam.io.gcp.pubsub import PubSubSource +from apache_beam.io.gcp.pubsub import _decodeUtf8String +from apache_beam.io.gcp.pubsub import _encodeUtf8String +from apache_beam.io.gcp.pubsub import _PubSubPayloadSink +from apache_beam.io.gcp.pubsub import _PubSubPayloadSource +from apache_beam.io.gcp.pubsub import ReadStringsFromPubSub +from apache_beam.io.gcp.pubsub import WriteStringsToPubSub +from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher -class TestPubSubSource(unittest.TestCase): +class TestReadStringsFromPubSub(unittest.TestCase): + def test_expand(self): + p = TestPipeline() + pcoll = p | ReadStringsFromPubSub('a_topic', 'a_subscription', 'a_label') + # Ensure that the output type is str + self.assertEqual(unicode, pcoll.element_type) + + # Ensure that the type on the intermediate read output PCollection is bytes + read_pcoll = pcoll.producer.inputs[0] + self.assertEqual(bytes, read_pcoll.element_type) + + # Ensure that the properties passed through correctly + source = read_pcoll.producer.transform.source + self.assertEqual('a_topic', source.topic) + self.assertEqual('a_subscription', source.subscription) + self.assertEqual('a_label', source.id_label) + + +class TestWriteStringsToPubSub(unittest.TestCase): + def test_expand(self): + p = TestPipeline() + pdone = p | ReadStringsFromPubSub('baz') | WriteStringsToPubSub('a_topic') + + # Ensure that the properties passed through correctly + sink = pdone.producer.transform.sink + self.assertEqual('a_topic', sink.topic) + # Ensure that the type on the intermediate payload transformer output + # PCollection is bytes + write_pcoll = pdone.producer.inputs[0] + self.assertEqual(bytes, write_pcoll.element_type) + + +class TestPubSubSource(unittest.TestCase): def test_display_data(self): - source = PubSubSource('a_topic', 'a_subscription', 'a_label') + source = _PubSubPayloadSource('a_topic', 'a_subscription', 'a_label') dd = DisplayData.create_from(source) expected_items = [ DisplayDataItemMatcher('topic', 'a_topic'), @@ -41,7 +78,7 @@ def test_display_data(self): hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items)) def test_display_data_no_subscription(self): - source = PubSubSource('a_topic') + source = _PubSubPayloadSource('a_topic') dd = DisplayData.create_from(source) expected_items = [ DisplayDataItemMatcher('topic', 'a_topic')] @@ -51,7 +88,7 @@ def test_display_data_no_subscription(self): class TestPubSubSink(unittest.TestCase): def test_display_data(self): - sink = PubSubSink('a_topic') + sink = _PubSubPayloadSink('a_topic') dd = DisplayData.create_from(sink) expected_items = [ DisplayDataItemMatcher('topic', 'a_topic')] @@ -59,6 +96,14 @@ def test_display_data(self): hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items)) +class TestEncodeDecodeUtf8String(unittest.TestCase): + def test_encode(self): + self.assertEqual(b'test_data', _encodeUtf8String('test_data')) + + def test_decode(self): + self.assertEqual('test_data', _decodeUtf8String(b'test_data')) + + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) unittest.main() diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index a05e582b05f4..046d3d585388 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -595,8 +595,8 @@ def run_Read(self, transform_node): standard_options = ( transform_node.inputs[0].pipeline.options.view_as(StandardOptions)) if not standard_options.streaming: - raise ValueError('PubSubSource is currently available for use only in ' - 'streaming pipelines.') + raise ValueError('PubSubPayloadSource is currently available for use ' + 'only in streaming pipelines.') step.add_property(PropertyNames.PUBSUB_TOPIC, transform.source.topic) if transform.source.subscription: step.add_property(PropertyNames.PUBSUB_SUBSCRIPTION, @@ -677,8 +677,8 @@ def run__NativeWrite(self, transform_node): standard_options = ( transform_node.inputs[0].pipeline.options.view_as(StandardOptions)) if not standard_options.streaming: - raise ValueError('PubSubSink is currently available for use only in ' - 'streaming pipelines.') + raise ValueError('PubSubPayloadSink is currently available for use ' + 'only in streaming pipelines.') step.add_property(PropertyNames.PUBSUB_TOPIC, transform.sink.topic) else: raise ValueError( From 4e8c388a46636a8e5391517876ec1adb818b8d61 Mon Sep 17 00:00:00 2001 From: Borisa Zivkovic Date: Mon, 15 May 2017 08:56:19 +0100 Subject: [PATCH 096/159] [BEAM-2276] Add windowing into default filename policy --- .../beam/sdk/io/DefaultFilenamePolicy.java | 151 ++++++++++++++++-- .../java/org/apache/beam/sdk/io/TextIO.java | 8 +- .../sdk/io/DefaultFilenamePolicyTest.java | 57 +++++++ .../org/apache/beam/sdk/io/TextIOTest.java | 10 -- 4 files changed, 196 insertions(+), 30 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java index 07bc2db4d804..50738546b14e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java @@ -32,22 +32,58 @@ import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** - * A default {@link FilenamePolicy} for unwindowed files. This policy is constructed using three - * parameters that together define the output name of a sharded file, in conjunction with the number - * of shards and index of the particular file, using {@link #constructName}. + * A default {@link FilenamePolicy} for windowed and unwindowed files. This policy is constructed + * using three parameters that together define the output name of a sharded file, in conjunction + * with the number of shards, index of the particular file, current window and pane information, + * using {@link #constructName}. * - *

Most users of unwindowed files will use this {@link DefaultFilenamePolicy}. For more advanced + *

Most users will use this {@link DefaultFilenamePolicy}. For more advanced * uses in generating different files for each window and other sharding controls, see the * {@code WriteOneFilePerWindow} example pipeline. */ public final class DefaultFilenamePolicy extends FilenamePolicy { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultFilenamePolicy.class); + /** The default sharding name template used in {@link #constructUsingStandardParameters}. */ public static final String DEFAULT_SHARD_TEMPLATE = ShardNameTemplate.INDEX_OF_MAX; + /** The default windowed sharding name template used when writing windowed files. + * This is used as default in cases when user did not specify shard template to + * be used and there is a need to write windowed files. In cases when user does + * specify shard template to be used then provided template will be used for both + * windowed and non-windowed file names. + */ + private static final String DEFAULT_WINDOWED_SHARD_TEMPLATE = + "P-W" + DEFAULT_SHARD_TEMPLATE; + + /* + * pattern for only non-windowed file names + */ + private static final String NON_WINDOWED_ONLY_PATTERN = "S+|N+"; + + /* + * pattern for only windowed file names + */ + private static final String WINDOWED_ONLY_PATTERN = "P|W"; + + /* + * pattern for both windowed and non-windowed file names + */ + private static final String TEMPLATE_PATTERN = "(" + NON_WINDOWED_ONLY_PATTERN + "|" + + WINDOWED_ONLY_PATTERN + ")"; + // Pattern that matches shard placeholders within a shard template. - private static final Pattern SHARD_FORMAT_RE = Pattern.compile("(S+|N+)"); + private static final Pattern SHARD_FORMAT_RE = Pattern.compile(TEMPLATE_PATTERN); + private static final Pattern WINDOWED_FORMAT_RE = Pattern.compile(WINDOWED_ONLY_PATTERN); /** * Constructs a new {@link DefaultFilenamePolicy}. @@ -69,7 +105,8 @@ public final class DefaultFilenamePolicy extends FilenamePolicy { *

Any filename component of the provided resource will be used as the filename prefix. * *

If provided, the shard name template will be used; otherwise {@link #DEFAULT_SHARD_TEMPLATE} - * will be used. + * will be used for non-windowed file names and {@link #DEFAULT_WINDOWED_SHARD_TEMPLATE} will + * be used for windowed file names. * *

If provided, the suffix will be used; otherwise the files will have an empty suffix. */ @@ -87,6 +124,19 @@ public static DefaultFilenamePolicy constructUsingStandardParameters( private final String shardTemplate; private final String suffix; + /* + * Checks whether given template contains enough information to form + * meaningful windowed file names - ie whether it uses pane and window + * info. + */ + static boolean isWindowedTemplate(String template){ + if (template != null){ + Matcher m = WINDOWED_FORMAT_RE.matcher(template); + return m.find(); + } + return false; + } + /** * Constructs a fully qualified name from components. * @@ -95,29 +145,45 @@ public static DefaultFilenamePolicy constructUsingStandardParameters( * strings. * *

Within a shard template, repeating sequences of the letters "S" or "N" - * are replaced with the shard number, or number of shards respectively. The - * numbers are formatted with leading zeros to match the length of the + * are replaced with the shard number, or number of shards respectively. + * "P" is replaced with by stringification of current pane. + * "W" is replaced by stringification of current window. + * + *

The numbers are formatted with leading zeros to match the length of the * repeated sequence of letters. * *

For example, if prefix = "output", shardTemplate = "-SSS-of-NNN", and * suffix = ".txt", with shardNum = 1 and numShards = 100, the following is * produced: "output-001-of-100.txt". */ - public static String constructName( - String prefix, String shardTemplate, String suffix, int shardNum, int numShards) { + static String constructName( + String prefix, String shardTemplate, String suffix, int shardNum, int numShards, + String paneStr, String windowStr) { // Matcher API works with StringBuffer, rather than StringBuilder. StringBuffer sb = new StringBuffer(); sb.append(prefix); Matcher m = SHARD_FORMAT_RE.matcher(shardTemplate); while (m.find()) { - boolean isShardNum = (m.group(1).charAt(0) == 'S'); + boolean isCurrentShardNum = (m.group(1).charAt(0) == 'S'); + boolean isNumberOfShards = (m.group(1).charAt(0) == 'N'); + boolean isPane = (m.group(1).charAt(0) == 'P') && paneStr != null; + boolean isWindow = (m.group(1).charAt(0) == 'W') && windowStr != null; char[] zeros = new char[m.end() - m.start()]; Arrays.fill(zeros, '0'); DecimalFormat df = new DecimalFormat(String.valueOf(zeros)); - String formatted = df.format(isShardNum ? shardNum : numShards); - m.appendReplacement(sb, formatted); + if (isCurrentShardNum) { + String formatted = df.format(shardNum); + m.appendReplacement(sb, formatted); + } else if (isNumberOfShards) { + String formatted = df.format(numShards); + m.appendReplacement(sb, formatted); + } else if (isPane) { + m.appendReplacement(sb, paneStr); + } else if (isWindow) { + m.appendReplacement(sb, windowStr); + } } m.appendTail(sb); @@ -125,6 +191,11 @@ public static String constructName( return sb.toString(); } + static String constructName(String prefix, String shardTemplate, String suffix, int shardNum, + int numShards) { + return constructName(prefix, shardTemplate, suffix, shardNum, numShards, null, null); + } + @Override @Nullable public ResourceId unwindowedFilename(ResourceId outputDirectory, Context context, @@ -138,9 +209,57 @@ public ResourceId unwindowedFilename(ResourceId outputDirectory, Context context @Override public ResourceId windowedFilename(ResourceId outputDirectory, - WindowedContext c, String extension) { - throw new UnsupportedOperationException("There is no default policy for windowed file" - + " output. Please provide an explicit FilenamePolicy to generate filenames."); + WindowedContext context, String extension) { + + boolean shardTemplateProvidedByUser = !this.shardTemplate.equals(DEFAULT_SHARD_TEMPLATE); + + if (shardTemplateProvidedByUser){ + boolean isWindowed = isWindowedTemplate(this.shardTemplate); + if (!isWindowed){ + LOG.info("Template you provided {} does not have enough information to create" + + "meaningful windowed file names. Consider using P and W in your template", + this.shardTemplate); + } + } + + final PaneInfo paneInfo = context.getPaneInfo(); + String paneStr = paneInfoToString(paneInfo); + String windowStr = windowToString(context.getWindow()); + + String templateToUse = shardTemplate; + if (!shardTemplateProvidedByUser){ + LOG.info("User did not provide shard template. For creating windowed file names " + + "default template {} will be used", DEFAULT_WINDOWED_SHARD_TEMPLATE); + templateToUse = DEFAULT_WINDOWED_SHARD_TEMPLATE; + } + + String filename = constructName(prefix.get(), templateToUse, suffix, + context.getShardNumber(), context.getNumShards(), paneStr, windowStr) + + extension; + return outputDirectory.resolve(filename, StandardResolveOptions.RESOLVE_FILE); + } + + /* + * Since not all windows have toString() that is nice or is compatible to be a part of file name. + */ + private String windowToString(BoundedWindow window) { + if (window instanceof GlobalWindow) { + return "GlobalWindow"; + } + if (window instanceof IntervalWindow) { + IntervalWindow iw = (IntervalWindow) window; + return String.format("IntervalWindow-%s-%s", iw.start().toString(), + iw.end().toString()); + } + return window.toString(); + } + + private String paneInfoToString(PaneInfo paneInfo){ + long currentPaneIndex = (paneInfo == null ? -1L + : paneInfo.getIndex()); + boolean firstPane = (paneInfo == null ? false : paneInfo.isFirst()); + boolean lastPane = (paneInfo == null ? false : paneInfo.isLast()); + return String.format("pane-%s-%b-%b", currentPaneIndex, firstPane, lastPane); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index 5c068ce06441..afb5849ef7c0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -70,8 +70,10 @@ * {@link TextIO.Write#withWindowedWrites()} will cause windowing and triggering to be * preserved. When producing windowed writes, the number of output shards must be set explicitly * using {@link TextIO.Write#withNumShards(int)}; some runners may set this for you to a - * runner-chosen value, so you may need not set it yourself. A {@link FilenamePolicy} must be - * set, and unique windows and triggers must produce unique filenames. + * runner-chosen value, so you may need not set it yourself. A {@link FilenamePolicy} can also be + * set in case you need better control over naming files created by unique windows. + * {@link DefaultFilenamePolicy} policy for producing unique filenames might not be appropriate + * for your use case. * *

Any existing files with the same names as generated output files will be overwritten. * @@ -434,8 +436,6 @@ public PDone expand(PCollection input) { (getFilenamePolicy() == null) || (getShardTemplate() == null && getFilenameSuffix() == null), "Cannot set a filename policy and also a filename template or suffix."); - checkState(!getWindowedWrites() || (getFilenamePolicy() != null), - "When using windowed writes, a filename policy must be set via withFilenamePolicy()."); FilenamePolicy usedFilenamePolicy = getFilenamePolicy(); if (usedFilenamePolicy == null) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java index c895da87f718..787403bad308 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java @@ -18,7 +18,10 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.io.DefaultFilenamePolicy.constructName; +import static org.apache.beam.sdk.io.DefaultFilenamePolicy.isWindowedTemplate; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import org.junit.Test; import org.junit.runner.RunWith; @@ -29,6 +32,7 @@ */ @RunWith(JUnit4.class) public class DefaultFilenamePolicyTest { + @Test public void testConstructName() { assertEquals("output-001-of-123.txt", @@ -52,4 +56,57 @@ public void testConstructNameWithLargeShardCount() { assertEquals("out-100-of-5000.txt", constructName("out", "-SS-of-NN", ".txt", 100, 5000)); } + + @Test + public void testIsWindowedTemplate(){ + assertTrue(isWindowedTemplate("-SSS-of-NNN-P-W")); + assertTrue(isWindowedTemplate("-SSS-of-NNN-W")); + assertTrue(isWindowedTemplate("-SSS-of-NNN-P")); + assertTrue(isWindowedTemplate("W-SSS-of-NNN")); + + assertFalse(isWindowedTemplate("-SSS-of-NNN")); + assertFalse(isWindowedTemplate("-SSS-of-lp")); + } + + @Test + public void testConstructWindowedName() { + assertEquals("output-001-of-123.txt", + constructName("output", "-SSS-of-NNN", ".txt", 1, 123, null, null)); + + assertEquals("output-001-of-123-PPP-W.txt", + constructName("output", "-SSS-of-NNN-PPP-W", ".txt", 1, 123, null, null)); + + assertEquals("out.txt/part-00042-myPaneStr-myWindowStr", + constructName("out.txt", "/part-SSSSS-P-W", "", 42, 100, "myPaneStr", + "myWindowStr")); + + assertEquals("out.txt", constructName("ou", "t.t", "xt", 1, 1, "myPaneStr2", + "anotherWindowStr")); + + assertEquals("out0102shard-oneMoreWindowStr-anotherPaneStr.txt", + constructName("out", "SSNNshard-W-P", ".txt", 1, 2, "anotherPaneStr", + "oneMoreWindowStr")); + + assertEquals("out-2/1.part-1-of-2-slidingWindow1-myPaneStr3-windowslidingWindow1-" + + "panemyPaneStr3.txt", + constructName("out", "-N/S.part-S-of-N-W-P-windowW-paneP", ".txt", 1, 2, "myPaneStr3", + "slidingWindow1")); + + // test first/last pane + assertEquals("out.txt/part-00042-myWindowStr-pane-11-true-false", + constructName("out.txt", "/part-SSSSS-W-P", "", 42, 100, "pane-11-true-false", + "myWindowStr")); + + assertEquals("out.txt", constructName("ou", "t.t", "xt", 1, 1, "pane", + "anotherWindowStr")); + + assertEquals("out0102shard-oneMoreWindowStr-pane--1-false-false-pane--1-false-false.txt", + constructName("out", "SSNNshard-W-P-P", ".txt", 1, 2, "pane--1-false-false", + "oneMoreWindowStr")); + + assertEquals("out-2/1.part-1-of-2-sWindow1-winsWindow1-ppaneL.txt", + constructName("out", + "-N/S.part-S-of-N-W-winW-pP", ".txt", 1, 2, "paneL", "sWindow1")); + } + } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java index 0d8fbbd34fa3..6c7a53f87664 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java @@ -1103,15 +1103,5 @@ public void testInitialSplitGzipModeGz() throws Exception { SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options); } - @Test - public void testWindowedWriteRequiresFilenamePolicy() { - PCollection emptyInput = p.apply(Create.empty(StringUtf8Coder.of())); - TextIO.Write write = TextIO.write().to("/tmp/some/file").withWindowedWrites(); - - expectedException.expect(IllegalStateException.class); - expectedException.expectMessage( - "When using windowed writes, a filename policy must be set via withFilenamePolicy()"); - emptyInput.apply(write); - } } From de38410d3e2cf9c6edff9438d539929777ad7915 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B3=A2=E7=89=B9?= Date: Thu, 20 Apr 2017 20:27:31 +0800 Subject: [PATCH 097/159] fix FlinkAccumulatorCombiningStateWithContext read null accum bug --- .../wrappers/streaming/state/FlinkStateInternals.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java index 9cb742ee7e30..b73abe9b68b7 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java @@ -821,7 +821,11 @@ public OutputT read() { flinkStateDescriptor); AccumT accum = state.value(); - return combineFn.extractOutput(accum, context); + if (accum != null) { + return combineFn.extractOutput(accum, context); + } else { + return combineFn.extractOutput(combineFn.createAccumulator(context), context); + } } catch (Exception e) { throw new RuntimeException("Error reading state.", e); } From bd1dfdf3c8e145a99bcacebd0c64dcf6580f3ffe Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 23 May 2017 13:29:51 -0700 Subject: [PATCH 098/159] Visit a Transform Hierarchy in Topological Order Always ensure that the producer of a value is visited before that value is visited for the first time. Visit a compoosite before visiting any of its child nodes. --- .../beam/runners/spark/SparkRunner.java | 13 +++ .../spark/translation/BoundedDataset.java | 6 ++ .../translation/TransformTranslator.java | 1 + .../spark/translation/StorageLevelTest.java | 4 +- .../beam/sdk/runners/TransformHierarchy.java | 46 +++++++++- .../sdk/runners/TransformHierarchyTest.java | 86 +++++++++++++++++++ 6 files changed, 150 insertions(+), 6 deletions(-) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index 9e2426ef8381..1be5e13f4f5b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -21,8 +21,10 @@ import com.google.common.collect.Iterables; import java.util.Arrays; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -49,6 +51,7 @@ import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.TransformHierarchy; +import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PCollection; @@ -336,6 +339,8 @@ public static class Evaluator extends Pipeline.PipelineVisitor.Defaults { protected final EvaluationContext ctxt; protected final SparkPipelineTranslator translator; + private final Set shouldIgnoreChildren = new HashSet<>(); + public Evaluator(SparkPipelineTranslator translator, EvaluationContext ctxt) { this.translator = translator; this.ctxt = ctxt; @@ -351,6 +356,7 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { LOG.info("Entering directly-translatable composite transform: '{}'", node.getFullName()); LOG.debug("Composite transform class: '{}'", transformClass); doVisitTransform(node); + shouldIgnoreChildren.add(node); return CompositeBehavior.DO_NOT_ENTER_TRANSFORM; } } @@ -392,6 +398,13 @@ protected boolean shouldDefer(TransformHierarchy.Node node) { @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { + Node parent = node.getEnclosingNode(); + while (!parent.isRootNode()) { + if (shouldIgnoreChildren.contains(parent)) { + return; + } + parent = parent.getEnclosingNode(); + } doVisitTransform(node); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java index 652c75305f0c..a746634008ef 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java @@ -19,6 +19,7 @@ package org.apache.beam.runners.spark.translation; import com.google.common.base.Function; +import com.google.common.base.Joiner; import com.google.common.collect.Iterables; import java.util.List; import javax.annotation.Nullable; @@ -97,8 +98,13 @@ public WindowedValue apply(byte[] bytes) { return windowedValues; } + int timesCached = 0; @Override public void cache(String storageLevel) { + System.out.printf( + "Persisting Dataset %s for RDD %s (id %s) at level %s. %s times before%n", + this, getRDD(), getRDD().toDebugString(), storageLevel, timesCached++); + System.out.println(Joiner.on("\n\t").join(new Throwable().getStackTrace())); // populate the rdd if needed getRDD().persist(StorageLevel.fromString(storageLevel)); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 742ea83539e2..6ca12c9ca6cc 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -407,6 +407,7 @@ public void evaluate(Read.Bounded transform, EvaluationContext context) { JavaRDD> input = new SourceRDD.Bounded<>( jsc.sc(), transform.getSource(), runtimeContext, stepName).toJavaRDD(); // cache to avoid re-evaluation of the source by Spark's lazy DAG evaluation. + System.out.println("Evaluating Bounded Read " + transform); context.putDataset(transform, new BoundedDataset<>(input.cache())); } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java index 8f2e681c9e91..8bd6dae98a74 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java @@ -58,12 +58,12 @@ public static void teardown() { @Test public void test() throws Exception { - PCollection pCollection = pipeline.apply(Create.of("foo")); + PCollection pCollection = pipeline.apply("CreateFoo", Create.of("foo")); // by default, the Spark runner doesn't cache the RDD if it accessed only one time. // So, to "force" the caching of the RDD, we have to call the RDD at least two time. // That's why we are using Count fn on the PCollection. - pCollection.apply(Count.globally()); + pCollection.apply("CountAll", Count.globally()); PCollection output = pCollection.apply(new StorageLevelPTransform()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index 2f0e8efd7de8..630d24cf68f1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -202,10 +202,12 @@ Node getProducer(PValue produced) { return producers.get(produced); } + int traversed = 0; public Set visit(PipelineVisitor visitor) { finishSpecifying(); Set visitedValues = new HashSet<>(); - root.visit(visitor, visitedValues); + traversed++; + root.visit(visitor, visitedValues, new HashSet(), new HashSet()); return visitedValues; } @@ -462,7 +464,22 @@ public Map, PValue> getOutputs() { *

Provides an ordered visit of the input values, the primitive transform (or child nodes for * composite transforms), then the output values. */ - private void visit(PipelineVisitor visitor, Set visitedValues) { + private void visit( + PipelineVisitor visitor, + Set visitedValues, + Set visitedNodes, + Set passedComposites) { + if (getEnclosingNode() != null && !visitedNodes.contains(getEnclosingNode())) { + getEnclosingNode().visit(visitor, visitedValues, visitedNodes, passedComposites); + } + if (!visitedNodes.add(this)) { + LOG.debug("Not revisiting previously visited node {}", this); + return; + } else if (childNodeOf(passedComposites)) { + LOG.debug("Not revisiting Node {} which is a child of a previously passed composite", this); + return; + } + if (!finishedSpecifying) { finishSpecifying(); } @@ -470,22 +487,31 @@ private void visit(PipelineVisitor visitor, Set visitedValues) { if (!isRootNode()) { // Visit inputs. for (PValue inputValue : inputs.values()) { + Node valueProducer = getProducer(inputValue); + if (!visitedNodes.contains(valueProducer)) { + valueProducer.visit(visitor, visitedValues, visitedNodes, passedComposites); + } if (visitedValues.add(inputValue)) { - visitor.visitValue(inputValue, getProducer(inputValue)); + LOG.debug("Visiting input value {}", inputValue); + visitor.visitValue(inputValue, valueProducer); } } } if (isCompositeNode()) { + LOG.debug("Visiting composite node {}", this); PipelineVisitor.CompositeBehavior recurse = visitor.enterCompositeTransform(this); if (recurse.equals(CompositeBehavior.ENTER_TRANSFORM)) { for (Node child : parts) { - child.visit(visitor, visitedValues); + child.visit(visitor, visitedValues, visitedNodes, passedComposites); } + } else { + passedComposites.add(this); } visitor.leaveCompositeTransform(this); } else { + LOG.debug("Visiting primitive node {}", this); visitor.visitPrimitiveTransform(this); } @@ -494,12 +520,24 @@ private void visit(PipelineVisitor visitor, Set visitedValues) { // Visit outputs. for (PValue pValue : outputs.values()) { if (visitedValues.add(pValue)) { + LOG.debug("Visiting output value {}", pValue); visitor.visitValue(pValue, this); } } } } + private boolean childNodeOf(Set nodes) { + if (isRootNode()) { + return false; + } + Node parent = this.getEnclosingNode(); + while (!parent.isRootNode() && !nodes.contains(parent)) { + parent = parent.getEnclosingNode(); + } + return nodes.contains(parent); + } + /** * Finish specifying a transform. * diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java index 1197d1b04eb6..2fe2817f534c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java @@ -19,6 +19,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; @@ -32,6 +33,8 @@ import java.util.Set; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.Pipeline.PipelineVisitor.Defaults; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.Read; @@ -492,4 +495,87 @@ public void visitPrimitiveTransform(Node node) { assertThat(visitedPrimitiveNodes, containsInAnyOrder(upstreamNode, replacementParNode)); assertThat(visitedValues, Matchers.containsInAnyOrder(upstream, output)); } + + @Test + public void visitIsTopologicallyOrdered() { + PCollection one = + PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED) + .setCoder(StringUtf8Coder.of()); + final PCollection two = + PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) + .setCoder(VarIntCoder.of()); + final PDone done = PDone.in(pipeline); + final TupleTag oneTag = new TupleTag() {}; + final TupleTag twoTag = new TupleTag() {}; + final PCollectionTuple oneAndTwo = PCollectionTuple.of(oneTag, one).and(twoTag, two); + + hierarchy.pushNode("consumes_both", one, new PTransform, PDone>() { + @Override + public PDone expand(PCollection input) { + return done; + } + + @Override + public Map, PValue> getAdditionalInputs() { + return Collections., PValue>singletonMap(twoTag, two); + } + }); + hierarchy.setOutput(done); + hierarchy.popNode(); + + final PTransform producer = + new PTransform() { + @Override + public PCollectionTuple expand(PBegin input) { + return oneAndTwo; + } + }; + hierarchy.pushNode( + "encloses_producer", + PBegin.in(pipeline), + new PTransform() { + @Override + public PCollectionTuple expand(PBegin input) { + return input.apply(producer); + } + }); + hierarchy.pushNode( + "creates_one_and_two", + PBegin.in(pipeline), producer); + hierarchy.setOutput(oneAndTwo); + hierarchy.popNode(); + hierarchy.setOutput(oneAndTwo); + hierarchy.popNode(); + + hierarchy.visit(new PipelineVisitor.Defaults() { + private final Set visitedNodes = new HashSet<>(); + private final Set visitedValues = new HashSet<>(); + @Override + public CompositeBehavior enterCompositeTransform(Node node) { + for (PValue input : node.getInputs().values()) { + assertThat(visitedValues, hasItem(input)); + } + visitedNodes.add(node); + return CompositeBehavior.ENTER_TRANSFORM; + } + + @Override + public void visitPrimitiveTransform(Node node) { + assertThat(visitedNodes, hasItem(node.getEnclosingNode())); + for (PValue input : node.getInputs().values()) { + assertThat(visitedValues, hasItem(input)); + } + visitedNodes.add(node); + } + + @Override + public void visitValue(PValue value, Node producer) { + assertThat(visitedNodes, hasItem(producer)); + assertThat(visitedValues, not(hasItem(value))); + visitedValues.add(value); + } + }); + } } From 8eb09aad9c975f787ba8afac83394cc8b56eb94f Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 25 May 2017 10:41:56 -0700 Subject: [PATCH 099/159] Update Apex Overrides Only override CreatePCollectionView transforms --- .../apache/beam/runners/apex/ApexRunner.java | 119 ++++-------------- 1 file changed, 21 insertions(+), 98 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java index f91d8e550789..c595b3f50b2e 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java @@ -57,7 +57,6 @@ import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Combine.GloballyAsSingletonView; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -65,7 +64,7 @@ import org.apache.beam.sdk.transforms.ParDo.MultiOutput; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.View.AsIterable; -import org.apache.beam.sdk.transforms.View.AsSingleton; +import org.apache.beam.sdk.transforms.View.CreatePCollectionView; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; @@ -111,16 +110,12 @@ private List getOverrides() { new PrimitiveCreate.Factory())) .add( PTransformOverride.of( - PTransformMatchers.classEqualTo(View.AsSingleton.class), - new StreamingViewAsSingleton.Factory())) - .add( - PTransformOverride.of( - PTransformMatchers.classEqualTo(View.AsIterable.class), + PTransformMatchers.createViewWithViewFn(PCollectionViews.IterableViewFn.class), new StreamingViewAsIterable.Factory())) .add( PTransformOverride.of( - PTransformMatchers.classEqualTo(Combine.GloballyAsSingletonView.class), - new StreamingCombineGloballyAsSingletonView.Factory())) + PTransformMatchers.createViewWithViewFn(PCollectionViews.SingletonViewFn.class), + new StreamingWrapSingletonInList.Factory())) .add( PTransformOverride.of( PTransformMatchers.splittableParDoMulti(), @@ -245,117 +240,45 @@ public void processElement(ProcessContext c) { } } - private static class StreamingCombineGloballyAsSingletonView - extends PTransform, PCollectionView> { + private static class StreamingWrapSingletonInList + extends PTransform, PCollectionView> { private static final long serialVersionUID = 1L; - Combine.GloballyAsSingletonView transform; + CreatePCollectionView transform; /** * Builds an instance of this class from the overridden transform. */ - private StreamingCombineGloballyAsSingletonView( - Combine.GloballyAsSingletonView transform) { + private StreamingWrapSingletonInList( + CreatePCollectionView transform) { this.transform = transform; } @Override - public PCollectionView expand(PCollection input) { - PCollection combined = input - .apply(Combine.globally(transform.getCombineFn()) - .withoutDefaults().withFanout(transform.getFanout())); - - PCollectionView view = PCollectionViews.singletonView(combined, - combined.getWindowingStrategy(), transform.getInsertDefault(), - transform.getInsertDefault() ? transform.getCombineFn().defaultValue() : null, - combined.getCoder()); - return combined.apply(ParDo.of(new WrapAsList())) - .apply(CreateApexPCollectionView. of(view)); + public PCollectionView expand(PCollection input) { + return input + .apply(ParDo.of(new WrapAsList())) + .apply(CreateApexPCollectionView.of(transform.getView())); } @Override protected String getKindString() { - return "StreamingCombineGloballyAsSingletonView"; + return "StreamingWrapSingletonInList"; } - static class Factory + static class Factory extends SingleInputOutputOverrideFactory< - PCollection, PCollectionView, - Combine.GloballyAsSingletonView> { + PCollection, PCollectionView, + CreatePCollectionView> { @Override - public PTransformReplacement, PCollectionView> + public PTransformReplacement, PCollectionView> getReplacementTransform( AppliedPTransform< - PCollection, PCollectionView, - GloballyAsSingletonView> + PCollection, PCollectionView, + CreatePCollectionView> transform) { return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), - new StreamingCombineGloballyAsSingletonView<>(transform.getTransform())); - } - } - } - - private static class StreamingViewAsSingleton - extends PTransform, PCollectionView> { - private static final long serialVersionUID = 1L; - - private View.AsSingleton transform; - - public StreamingViewAsSingleton(View.AsSingleton transform) { - this.transform = transform; - } - - @Override - public PCollectionView expand(PCollection input) { - Combine.Globally combine = Combine - .globally(new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue())); - if (!transform.hasDefaultValue()) { - combine = combine.withoutDefaults(); - } - return input.apply(combine.asSingletonView()); - } - - @Override - protected String getKindString() { - return "StreamingViewAsSingleton"; - } - - private static class SingletonCombine extends Combine.BinaryCombineFn { - private boolean hasDefaultValue; - private T defaultValue; - - SingletonCombine(boolean hasDefaultValue, T defaultValue) { - this.hasDefaultValue = hasDefaultValue; - this.defaultValue = defaultValue; - } - - @Override - public T apply(T left, T right) { - throw new IllegalArgumentException("PCollection with more than one element " - + "accessed as a singleton view. Consider using Combine.globally().asSingleton() to " - + "combine the PCollection into a single value"); - } - - @Override - public T identity() { - if (hasDefaultValue) { - return defaultValue; - } else { - throw new IllegalArgumentException("Empty PCollection accessed as a singleton view. " - + "Consider setting withDefault to provide a default value"); - } - } - } - - static class Factory - extends SingleInputOutputOverrideFactory< - PCollection, PCollectionView, View.AsSingleton> { - @Override - public PTransformReplacement, PCollectionView> getReplacementTransform( - AppliedPTransform, PCollectionView, AsSingleton> transform) { - return PTransformReplacement.of( - PTransformReplacements.getSingletonMainInput(transform), - new StreamingViewAsSingleton<>(transform.getTransform())); + new StreamingWrapSingletonInList<>(transform.getTransform())); } } } From 2e2ae9cfa581a73864695d15102acadc2750a57a Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 23 May 2017 09:43:45 -0700 Subject: [PATCH 100/159] Roll-forward Include Additional PTransform inputs in Transform Nodes Add the value of PTransform.getAdditionalInputs in the inputs of a TransformHierarchy node. Fork the Node constructor to reduce nullability This slightly simplifies the constructor implementation(s). --- .../apex/translation/TranslationContext.java | 4 +- .../core/construction/TransformInputs.java | 50 ++++++ .../construction/TransformInputsTest.java | 166 ++++++++++++++++++ .../runners/direct/DirectGraphVisitor.java | 15 +- .../runners/direct/ParDoEvaluatorFactory.java | 9 +- ...ttableProcessElementsEvaluatorFactory.java | 2 + .../direct/StatefulParDoEvaluatorFactory.java | 1 + .../beam/runners/direct/WatermarkManager.java | 17 +- .../runners/direct/ParDoEvaluatorTest.java | 6 +- .../flink/FlinkBatchTranslationContext.java | 3 +- .../FlinkStreamingTranslationContext.java | 3 +- .../dataflow/DataflowPipelineTranslator.java | 5 +- .../spark/translation/EvaluationContext.java | 4 +- .../beam/sdk/runners/TransformHierarchy.java | 28 ++- 14 files changed, 284 insertions(+), 29 deletions(-) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java index aff3863624c4..94d13e177dec 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java @@ -34,6 +34,7 @@ import org.apache.beam.runners.apex.translation.utils.ApexStateInternals.ApexStateBackend; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translation.utils.CoderAdapterStreamCodec; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -93,7 +94,8 @@ public Map, PValue> getInputs() { } public InputT getInput() { - return (InputT) Iterables.getOnlyElement(getCurrentTransform().getInputs().values()); + return (InputT) + Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(getCurrentTransform())); } public Map, PValue> getOutputs() { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java new file mode 100644 index 000000000000..2baf93a3c128 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.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.core.construction; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.common.collect.ImmutableList; +import java.util.Collection; +import java.util.Map; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; + +/** Utilities for extracting subsets of inputs from an {@link AppliedPTransform}. */ +public class TransformInputs { + /** + * Gets all inputs of the {@link AppliedPTransform} that are not returned by {@link + * PTransform#getAdditionalInputs()}. + */ + public static Collection nonAdditionalInputs(AppliedPTransform application) { + ImmutableList.Builder mainInputs = ImmutableList.builder(); + PTransform transform = application.getTransform(); + for (Map.Entry, PValue> input : application.getInputs().entrySet()) { + if (!transform.getAdditionalInputs().containsKey(input.getKey())) { + mainInputs.add(input.getValue()); + } + } + checkArgument( + !mainInputs.build().isEmpty() || application.getInputs().isEmpty(), + "Expected at least one main input if any inputs exist"); + return mainInputs.build(); + } +} diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java new file mode 100644 index 000000000000..f5b2c11e7923 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import static org.junit.Assert.assertThat; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.PTransform; +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.POutput; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.hamcrest.Matchers; +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 TransformInputs}. */ +@RunWith(JUnit4.class) +public class TransformInputsTest { + @Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false); + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void nonAdditionalInputsWithNoInputSucceeds() { + AppliedPTransform transform = + AppliedPTransform.of( + "input-free", + Collections., PValue>emptyMap(), + Collections., PValue>emptyMap(), + new TestTransform(), + pipeline); + + assertThat(TransformInputs.nonAdditionalInputs(transform), Matchers.empty()); + } + + @Test + public void nonAdditionalInputsWithOneMainInputSucceeds() { + PCollection input = pipeline.apply(GenerateSequence.from(1L)); + AppliedPTransform transform = + AppliedPTransform.of( + "input-single", + Collections., PValue>singletonMap(new TupleTag() {}, input), + Collections., PValue>emptyMap(), + new TestTransform(), + pipeline); + + assertThat( + TransformInputs.nonAdditionalInputs(transform), Matchers.containsInAnyOrder(input)); + } + + @Test + public void nonAdditionalInputsWithMultipleNonAdditionalInputsSucceeds() { + Map, PValue> allInputs = new HashMap<>(); + PCollection mainInts = pipeline.apply("MainInput", Create.of(12, 3)); + allInputs.put(new TupleTag() {}, mainInts); + PCollection voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of())); + allInputs.put(new TupleTag() {}, voids); + AppliedPTransform transform = + AppliedPTransform.of( + "additional-free", + allInputs, + Collections., PValue>emptyMap(), + new TestTransform(), + pipeline); + + assertThat( + TransformInputs.nonAdditionalInputs(transform), + Matchers.containsInAnyOrder(voids, mainInts)); + } + + @Test + public void nonAdditionalInputsWithAdditionalInputsSucceeds() { + Map, PValue> additionalInputs = new HashMap<>(); + additionalInputs.put(new TupleTag() {}, pipeline.apply(Create.of("1, 2", "3"))); + additionalInputs.put(new TupleTag() {}, pipeline.apply(GenerateSequence.from(3L))); + + Map, PValue> allInputs = new HashMap<>(); + PCollection mainInts = pipeline.apply("MainInput", Create.of(12, 3)); + allInputs.put(new TupleTag() {}, mainInts); + PCollection voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of())); + allInputs.put( + new TupleTag() {}, voids); + allInputs.putAll(additionalInputs); + + AppliedPTransform transform = + AppliedPTransform.of( + "additional", + allInputs, + Collections., PValue>emptyMap(), + new TestTransform(additionalInputs), + pipeline); + + assertThat( + TransformInputs.nonAdditionalInputs(transform), + Matchers.containsInAnyOrder(mainInts, voids)); + } + + @Test + public void nonAdditionalInputsWithOnlyAdditionalInputsThrows() { + Map, PValue> additionalInputs = new HashMap<>(); + additionalInputs.put(new TupleTag() {}, pipeline.apply(Create.of("1, 2", "3"))); + additionalInputs.put(new TupleTag() {}, pipeline.apply(GenerateSequence.from(3L))); + + AppliedPTransform transform = + AppliedPTransform.of( + "additional-only", + additionalInputs, + Collections., PValue>emptyMap(), + new TestTransform(additionalInputs), + pipeline); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("at least one"); + TransformInputs.nonAdditionalInputs(transform); + } + + private static class TestTransform extends PTransform { + private final Map, PValue> additionalInputs; + + private TestTransform() { + this(Collections., PValue>emptyMap()); + } + + private TestTransform(Map, PValue> additionalInputs) { + this.additionalInputs = additionalInputs; + } + + @Override + public POutput expand(PInput input) { + return PDone.in(input.getPipeline()); + } + + @Override + public Map, PValue> getAdditionalInputs() { + return additionalInputs; + } + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index 01204e3049dd..ed4282bbdb8b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -21,10 +21,12 @@ import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ListMultimap; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -34,6 +36,8 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the @@ -41,6 +45,7 @@ * input after the upstream transform has produced and committed output. */ class DirectGraphVisitor extends PipelineVisitor.Defaults { + private static final Logger LOG = LoggerFactory.getLogger(DirectGraphVisitor.class); private Map> producers = new HashMap<>(); @@ -83,7 +88,15 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (node.getInputs().isEmpty()) { rootTransforms.add(appliedTransform); } else { - for (PValue value : node.getInputs().values()) { + Collection mainInputs = + TransformInputs.nonAdditionalInputs(node.toAppliedPTransform(getPipeline())); + if (!mainInputs.containsAll(node.getInputs().values())) { + LOG.debug( + "Inputs reduced to {} from {} by removing additional inputs", + mainInputs, + node.getInputs().values()); + } + for (PValue value : mainInputs) { primitiveConsumers.put(value, appliedTransform); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index 8aa75cf1445e..516f798aba97 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -20,7 +20,6 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; -import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -79,6 +78,7 @@ public TransformEvaluator forApplication( (TransformEvaluator) createEvaluator( (AppliedPTransform) application, + (PCollection) inputBundle.getPCollection(), inputBundle.getKey(), doFn, transform.getSideInputs(), @@ -102,6 +102,7 @@ public void cleanup() throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( AppliedPTransform, PCollectionTuple, ?> application, + PCollection mainInput, StructuralKey inputBundleKey, DoFn doFn, List> sideInputs, @@ -120,6 +121,7 @@ DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( createParDoEvaluator( application, inputBundleKey, + mainInput, sideInputs, mainOutputTag, additionalOutputTags, @@ -132,6 +134,7 @@ DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( ParDoEvaluator createParDoEvaluator( AppliedPTransform, PCollectionTuple, ?> application, StructuralKey key, + PCollection mainInput, List> sideInputs, TupleTag mainOutputTag, List> additionalOutputTags, @@ -144,8 +147,7 @@ ParDoEvaluator createParDoEvaluator( evaluationContext, stepContext, application, - ((PCollection) Iterables.getOnlyElement(application.getInputs().values())) - .getWindowingStrategy(), + mainInput.getWindowingStrategy(), fn, key, sideInputs, @@ -173,5 +175,4 @@ static Map, PCollection> pcollections(Map, PValue> ou } return pcs; } - } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index b85f481c1489..eccc83a031cb 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -116,6 +116,8 @@ public void cleanup() throws Exception { delegateFactory.createParDoEvaluator( application, inputBundle.getKey(), + (PCollection>>) + inputBundle.getPCollection(), transform.getSideInputs(), transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 506c84cec639..3619d05b47ca 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -117,6 +117,7 @@ private TransformEvaluator>> createEvaluator( DoFnLifecycleManagerRemovingTransformEvaluator> delegateEvaluator = delegateFactory.createEvaluator( (AppliedPTransform) application, + (PCollection) inputBundle.getPCollection(), inputBundle.getKey(), doFn, application.getTransform().getUnderlyingParDo().getSideInputs(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index 4f1b8319dc2d..b15b52e314de 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -823,10 +823,11 @@ private Collection getInputProcessingWatermarks(AppliedPTransform getInputWatermarks(AppliedPTransform transform) inputWatermarksBuilder.add(THE_END_OF_TIME); } for (PValue pvalue : inputs.values()) { - Watermark producerOutputWatermark = - getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; - inputWatermarksBuilder.add(producerOutputWatermark); + if (graph.getPrimitiveConsumers(pvalue).contains(transform)) { + Watermark producerOutputWatermark = + getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; + inputWatermarksBuilder.add(producerOutputWatermark); + } } List inputCollectionWatermarks = inputWatermarksBuilder.build(); return inputCollectionWatermarks; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index 09a21ac524a5..8b86bbe50958 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -98,7 +98,7 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { when(evaluationContext.createBundle(output)).thenReturn(outputBundle); ParDoEvaluator evaluator = - createEvaluator(singletonView, fn, output); + createEvaluator(singletonView, fn, inputPc, output); IntervalWindow nonGlobalWindow = new IntervalWindow(new Instant(0), new Instant(10_000L)); WindowedValue first = WindowedValue.valueInGlobalWindow(3); @@ -132,6 +132,7 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { private ParDoEvaluator createEvaluator( PCollectionView singletonView, RecorderFn fn, + PCollection input, PCollection output) { when( evaluationContext.createSideInputReader( @@ -156,8 +157,7 @@ private ParDoEvaluator createEvaluator( evaluationContext, stepContext, transform, - ((PCollection) Iterables.getOnlyElement(transform.getInputs().values())) - .getWindowingStrategy(), + input.getWindowingStrategy(), fn, null /* key */, ImmutableList.>of(singletonView), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java index 0439119dfc40..6e7019848b19 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java @@ -20,6 +20,7 @@ import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.Map; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -143,7 +144,7 @@ Map, PValue> getInputs(PTransform transform) { @SuppressWarnings("unchecked") T getInput(PTransform transform) { - return (T) Iterables.getOnlyElement(currentTransform.getInputs().values()); + return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform)); } Map, PValue> getOutputs(PTransform transform) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java index ea5f6b3162af..74a5fb971144 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java @@ -22,6 +22,7 @@ import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.Map; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -113,7 +114,7 @@ public TypeInformation> getTypeInfo(PCollection collecti @SuppressWarnings("unchecked") public T getInput(PTransform transform) { - return (T) Iterables.getOnlyElement(currentTransform.getInputs().values()); + return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform)); } public Map, PValue> getInputs(PTransform transform) { 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 index af93ef59bba4..fccd018b13d2 100644 --- 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 @@ -56,6 +56,7 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import javax.annotation.Nullable; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.runners.dataflow.BatchViewOverrides.GroupByKeyAndSortValuesOnly; import org.apache.beam.runners.dataflow.DataflowRunner.CombineGroupedValues; @@ -395,7 +396,9 @@ public Map, PValue> getInputs( @Override public InputT getInput(PTransform transform) { - return (InputT) Iterables.getOnlyElement(getInputs(transform).values()); + return (InputT) + Iterables.getOnlyElement( + TransformInputs.nonAdditionalInputs(getCurrentTransform(transform))); } @Override diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index 8102926f6daa..0c6c4d1cb660 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -26,6 +26,7 @@ import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.sdk.Pipeline; @@ -103,7 +104,8 @@ public void setCurrentTransform(AppliedPTransform transform) { public T getInput(PTransform transform) { @SuppressWarnings("unchecked") - T input = (T) Iterables.getOnlyElement(getInputs(transform).values()); + T input = + (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(getCurrentTransform())); return input; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index 630d24cf68f1..bebc306a8d24 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -32,7 +32,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; @@ -68,7 +67,7 @@ public TransformHierarchy() { producers = new HashMap<>(); producerInput = new HashMap<>(); unexpandedInputs = new HashMap<>(); - root = new Node(null, null, "", null); + root = new Node(); current = root; } @@ -254,26 +253,37 @@ public class Node { @VisibleForTesting boolean finishedSpecifying = false; + /** + * Creates the root-level node. The root level node has a null enclosing node, a null transform, + * an empty map of inputs, and a name equal to the empty string. + */ + private Node() { + this.enclosingNode = null; + this.transform = null; + this.fullName = ""; + this.inputs = Collections.emptyMap(); + } + /** * Creates a new Node with the given parent and transform. * - *

EnclosingNode and transform may both be null for a root-level node, which holds all other - * nodes. - * * @param enclosingNode the composite node containing this node * @param transform the PTransform tracked by this node * @param fullName the fully qualified name of the transform * @param input the unexpanded input to the transform */ private Node( - @Nullable Node enclosingNode, - @Nullable PTransform transform, + Node enclosingNode, + PTransform transform, String fullName, - @Nullable PInput input) { + PInput input) { this.enclosingNode = enclosingNode; this.transform = transform; this.fullName = fullName; - this.inputs = input == null ? Collections., PValue>emptyMap() : input.expand(); + ImmutableMap.Builder, PValue> inputs = ImmutableMap.builder(); + inputs.putAll(input.expand()); + inputs.putAll(transform.getAdditionalInputs()); + this.inputs = inputs.build(); } /** From c8478fe1fe107b842d3cfa56b652d740fdf0c18b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 25 May 2017 07:25:08 -0700 Subject: [PATCH 101/159] Mark CombineFnWithContext StateSpecs internal --- .../java/org/apache/beam/sdk/state/StateSpecs.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java index 7b7138489997..5a2a1b6b9c72 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java @@ -76,7 +76,9 @@ StateSpec> combining( } /** - * Create a {@link StateSpec} for a {@link CombiningState} which uses a {@link + * For internal use only; no backwards compatibility guarantees + * + *

Create a {@link StateSpec} for a {@link CombiningState} which uses a {@link * CombineFnWithContext} to automatically merge multiple values of type {@code InputT} into a * single resulting {@code OutputT}. * @@ -84,6 +86,7 @@ StateSpec> combining( * * @see #combining(Coder, CombineFnWithContext) */ + @Internal public static StateSpec> combining( CombineFnWithContext combineFn) { @@ -105,11 +108,14 @@ StateSpec> combining( } /** - * Identical to {@link #combining(CombineFnWithContext)}, but with an accumulator coder explicitly - * supplied. + * For internal use only; no backwards compatibility guarantees + * + *

Identical to {@link #combining(CombineFnWithContext)}, but with an accumulator coder + * explicitly supplied. * *

If automatic coder inference fails, use this method. */ + @Internal public static StateSpec> combining( Coder accumCoder, CombineFnWithContext combineFn) { From b0dc523c72a68e870392fbac8ff9f3a87459ab22 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 25 May 2017 13:02:15 -0700 Subject: [PATCH 102/159] Allow translation to throw IOException --- .../beam/runners/core/construction/PTransformTranslation.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index 9f5f3b50b723..00ea55e6e1ed 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -138,7 +138,8 @@ public static String urnForTransform(PTransform transform) { */ public interface TransformPayloadTranslator> { String getUrn(T transform); - FunctionSpec translate(AppliedPTransform application, SdkComponents components); + FunctionSpec translate(AppliedPTransform application, SdkComponents components) + throws IOException; } /** From 9497e5eaecf5d7eb7f18709935c183b03116f75f Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 25 May 2017 07:12:08 -0700 Subject: [PATCH 103/159] Flesh out TimerSpec and StateSpec in Runner API --- .../src/main/proto/beam_runner_api.proto | 40 ++++++++++++++----- 1 file changed, 29 insertions(+), 11 deletions(-) diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto index c8722e6a39fb..16122093c812 100644 --- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto +++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto @@ -247,21 +247,39 @@ message Parameter { } message StateSpec { - // TODO: AST for state spec - string id = 1; - Type type = 2; - - enum Type { - VALUE = 0; - BAG = 1; - MAP = 2; - SET = 3; + oneof spec { + ValueStateSpec value_spec = 1; + BagStateSpec bag_spec = 2; + CombiningStateSpec combining_spec = 3; + MapStateSpec map_spec = 4; + SetStateSpec set_spec = 5; } } +message ValueStateSpec { + string coder_id = 1; +} + +message BagStateSpec { + string element_coder_id = 1; +} + +message CombiningStateSpec { + string accumulator_coder_id = 1; + SdkFunctionSpec combine_fn = 2; +} + +message MapStateSpec { + string key_coder_id = 1; + string value_coder_id = 2; +} + +message SetStateSpec { + string element_coder_id = 1; +} + message TimerSpec { - // TODO: AST for timer spec - string id = 1; + TimeDomain time_domain = 1; } enum IsBounded { From 8fc2eb0aeee9c3bdeaf93897e5e8aa4bb98b98de Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 25 May 2017 07:27:52 -0700 Subject: [PATCH 104/159] Add case dispatch to StateSpec This is different than a StateBinder: for a binder, the id is needed and the StateSpec controls the return type. For case dispatch, the dispatcher controls the type and it should just be reading the spec, which does not require the id. Eventually, StateBinder could be removed in favor of StateSpec.Cases>. --- .../org/apache/beam/sdk/state/StateSpec.java | 53 +++++++++++++++++++ .../org/apache/beam/sdk/state/StateSpecs.java | 41 ++++++++++++++ 2 files changed, 94 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpec.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpec.java index b0412bf49c6d..0443f25f7c4e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpec.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpec.java @@ -22,6 +22,7 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.Combine; /** * A specification of a persistent state cell. This includes information necessary to encode the @@ -40,6 +41,14 @@ public interface StateSpec extends Serializable { @Internal StateT bind(String id, StateBinder binder); + /** + * For internal use only; no backwards-compatibility guarantees. + * + *

Perform case analysis on this {@link StateSpec} using the provided {@link Cases}. + */ + @Internal + ResultT match(Cases cases); + /** * For internal use only; no backwards-compatibility guarantees. * @@ -60,4 +69,48 @@ public interface StateSpec extends Serializable { */ @Internal void finishSpecifying(); + + /** + * Cases for doing a "switch" on the type of {@link StateSpec}. + */ + interface Cases { + ResultT dispatchValue(Coder valueCoder); + ResultT dispatchBag(Coder elementCoder); + ResultT dispatchCombining(Combine.CombineFn combineFn, Coder accumCoder); + ResultT dispatchMap(Coder keyCoder, Coder valueCoder); + ResultT dispatchSet(Coder elementCoder); + + /** + * A base class for a visitor with a default method for cases it is not interested in. + */ + abstract class WithDefault implements Cases { + + protected abstract ResultT dispatchDefault(); + + @Override + public ResultT dispatchValue(Coder valueCoder) { + return dispatchDefault(); + } + + @Override + public ResultT dispatchBag(Coder elementCoder) { + return dispatchDefault(); + } + + @Override + public ResultT dispatchCombining(Combine.CombineFn combineFn, Coder accumCoder) { + return dispatchDefault(); + } + + @Override + public ResultT dispatchMap(Coder keyCoder, Coder valueCoder) { + return dispatchDefault(); + } + + @Override + public ResultT dispatchSet(Coder elementCoder) { + return dispatchDefault(); + } + } + } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java index 5a2a1b6b9c72..42223047cc58 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java @@ -278,6 +278,11 @@ public ValueState bind(String id, StateBinder visitor) { return visitor.bindValue(id, this, coder); } + @Override + public ResultT match(Cases cases) { + return cases.dispatchValue(coder); + } + @SuppressWarnings("unchecked") @Override public void offerCoders(Coder[] coders) { @@ -342,6 +347,11 @@ public CombiningState bind( return visitor.bindCombining(id, this, accumCoder, combineFn); } + @Override + public ResultT match(Cases cases) { + return cases.dispatchCombining(combineFn, accumCoder); + } + @SuppressWarnings("unchecked") @Override public void offerCoders(Coder[] coders) { @@ -413,6 +423,14 @@ public CombiningState bind( return visitor.bindCombiningWithContext(id, this, accumCoder, combineFn); } + @Override + public ResultT match(Cases cases) { + throw new UnsupportedOperationException( + String.format( + "%s is for internal use only and does not support case dispatch", + getClass().getSimpleName())); + } + @SuppressWarnings("unchecked") @Override public void offerCoders(Coder[] coders) { @@ -480,6 +498,11 @@ public BagState bind(String id, StateBinder visitor) { return visitor.bindBag(id, this, elemCoder); } + @Override + public ResultT match(Cases cases) { + return cases.dispatchBag(elemCoder); + } + @SuppressWarnings("unchecked") @Override public void offerCoders(Coder[] coders) { @@ -536,6 +559,11 @@ public MapState bind(String id, StateBinder visitor) { return visitor.bindMap(id, this, keyCoder, valueCoder); } + @Override + public ResultT match(Cases cases) { + return cases.dispatchMap(keyCoder, valueCoder); + } + @SuppressWarnings("unchecked") @Override public void offerCoders(Coder[] coders) { @@ -600,6 +628,11 @@ public SetState bind(String id, StateBinder visitor) { return visitor.bindSet(id, this, elemCoder); } + @Override + public ResultT match(Cases cases) { + return cases.dispatchSet(elemCoder); + } + @SuppressWarnings("unchecked") @Override public void offerCoders(Coder[] coders) { @@ -663,6 +696,14 @@ public WatermarkHoldState bind(String id, StateBinder visitor) { return visitor.bindWatermark(id, this, timestampCombiner); } + @Override + public ResultT match(Cases cases) { + throw new UnsupportedOperationException( + String.format( + "%s is for internal use only and does not support case dispatch", + getClass().getSimpleName())); + } + @Override public void offerCoders(Coder[] coders) { } From a250ce58c6a0caf473842c4e5e6f980a828dde55 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 25 May 2017 22:51:18 -0700 Subject: [PATCH 105/159] Make Java serialized CombineFn URN public --- .../beam/runners/core/construction/CombineTranslation.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java index 855fba740a02..28bc9a15e47f 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java @@ -49,7 +49,7 @@ * RunnerApi.CombinePayload} protos. */ public class CombineTranslation { - private static final String JAVA_SERIALIZED_COMBINE_FN_URN = "urn:beam:java:combinefn:v1"; + public static final String JAVA_SERIALIZED_COMBINE_FN_URN = "urn:beam:java:combinefn:v1"; public static CombinePayload toProto( AppliedPTransform> combine, SdkComponents sdkComponents) From 39220dbca944a2496587c543de2a4eb01004bd76 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 25 May 2017 07:12:29 -0700 Subject: [PATCH 106/159] Implement TimerSpec and StateSpec translation --- .../core/construction/CombineTranslation.java | 2 +- .../core/construction/ParDoTranslation.java | 215 +++++++++++++++-- .../construction/ParDoTranslationTest.java | 218 +++++++++++------- 3 files changed, 343 insertions(+), 92 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java index 28bc9a15e47f..472b6f8b1d48 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java @@ -86,7 +86,7 @@ private static Coder extractAccumulatorCoder( .getAccumulatorCoder(); } - private static SdkFunctionSpec toProto(GlobalCombineFn combineFn) { + public static SdkFunctionSpec toProto(GlobalCombineFn combineFn) { return SdkFunctionSpec.newBuilder() // TODO: Set Java SDK Environment URN .setSpec( diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java index 1c81f8ce05d5..fe66179bc5e7 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java @@ -19,10 +19,12 @@ package org.apache.beam.runners.core.construction; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN; import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; @@ -46,9 +48,12 @@ import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; import org.apache.beam.sdk.common.runner.v1.RunnerApi.SideInput; import org.apache.beam.sdk.common.runner.v1.RunnerApi.SideInput.Builder; -import org.apache.beam.sdk.common.runner.v1.RunnerApi.StateSpec; -import org.apache.beam.sdk.common.runner.v1.RunnerApi.TimerSpec; import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Materializations; import org.apache.beam.sdk.transforms.PTransform; @@ -107,7 +112,8 @@ public String getUrn(ParDo.MultiOutput transform) { @Override public FunctionSpec translate( - AppliedPTransform> transform, SdkComponents components) { + AppliedPTransform> transform, SdkComponents components) + throws IOException { ParDoPayload payload = toProto(transform.getTransform(), components); return RunnerApi.FunctionSpec.newBuilder() .setUrn(PAR_DO_TRANSFORM_URN) @@ -128,8 +134,10 @@ public static class Registrar implements TransformPayloadTranslatorRegistrar { } } - public static ParDoPayload toProto(ParDo.MultiOutput parDo, SdkComponents components) { - DoFnSignature signature = DoFnSignatures.getSignature(parDo.getFn().getClass()); + public static ParDoPayload toProto(ParDo.MultiOutput parDo, SdkComponents components) + throws IOException { + DoFn doFn = parDo.getFn(); + DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass()); Map states = signature.stateDeclarations(); Map timers = signature.timerDeclarations(); List parameters = signature.processElement().extraParameters(); @@ -146,16 +154,62 @@ public static ParDoPayload toProto(ParDo.MultiOutput parDo, SdkComponents } } for (Map.Entry state : states.entrySet()) { - StateSpec spec = toProto(state.getValue()); + RunnerApi.StateSpec spec = + toProto(getStateSpecOrCrash(state.getValue(), doFn), components); builder.putStateSpecs(state.getKey(), spec); } for (Map.Entry timer : timers.entrySet()) { - TimerSpec spec = toProto(timer.getValue()); + RunnerApi.TimerSpec spec = + toProto(getTimerSpecOrCrash(timer.getValue(), doFn)); builder.putTimerSpecs(timer.getKey(), spec); } return builder.build(); } + private static StateSpec getStateSpecOrCrash( + StateDeclaration stateDeclaration, DoFn target) { + try { + Object fieldValue = stateDeclaration.field().get(target); + checkState(fieldValue instanceof StateSpec, + "Malformed %s class %s: state declaration field %s does not have type %s.", + DoFn.class.getSimpleName(), + target.getClass().getName(), + stateDeclaration.field().getName(), + StateSpec.class); + + return (StateSpec) stateDeclaration.field().get(target); + } catch (IllegalAccessException exc) { + throw new RuntimeException( + String.format( + "Malformed %s class %s: state declaration field %s is not accessible.", + DoFn.class.getSimpleName(), + target.getClass().getName(), + stateDeclaration.field().getName())); + } + } + + private static TimerSpec getTimerSpecOrCrash( + TimerDeclaration timerDeclaration, DoFn target) { + try { + Object fieldValue = timerDeclaration.field().get(target); + checkState(fieldValue instanceof TimerSpec, + "Malformed %s class %s: timer declaration field %s does not have type %s.", + DoFn.class.getSimpleName(), + target.getClass().getName(), + timerDeclaration.field().getName(), + TimerSpec.class); + + return (TimerSpec) timerDeclaration.field().get(target); + } catch (IllegalAccessException exc) { + throw new RuntimeException( + String.format( + "Malformed %s class %s: timer declaration field %s is not accessible.", + DoFn.class.getSimpleName(), + target.getClass().getName(), + timerDeclaration.field().getName())); + } + } + public static DoFn getDoFn(ParDoPayload payload) throws InvalidProtocolBufferException { return doFnAndMainOutputTagFromProto(payload.getDoFn()).getDoFn(); } @@ -179,14 +233,149 @@ public static RunnerApi.PCollection getMainInput( return components.getPcollectionsOrThrow(ptransform.getInputsOrThrow(mainInputId)); } - // TODO: Implement - private static StateSpec toProto(StateDeclaration state) { - throw new UnsupportedOperationException("Not yet supported"); + @VisibleForTesting + static RunnerApi.StateSpec toProto(StateSpec stateSpec, final SdkComponents components) + throws IOException { + final RunnerApi.StateSpec.Builder builder = RunnerApi.StateSpec.newBuilder(); + + return stateSpec.match( + new StateSpec.Cases() { + @Override + public RunnerApi.StateSpec dispatchValue(Coder valueCoder) { + return builder + .setValueSpec( + RunnerApi.ValueStateSpec.newBuilder() + .setCoderId(registerCoderOrThrow(components, valueCoder))) + .build(); + } + + @Override + public RunnerApi.StateSpec dispatchBag(Coder elementCoder) { + return builder + .setBagSpec( + RunnerApi.BagStateSpec.newBuilder() + .setElementCoderId(registerCoderOrThrow(components, elementCoder))) + .build(); + } + + @Override + public RunnerApi.StateSpec dispatchCombining( + Combine.CombineFn combineFn, Coder accumCoder) { + return builder + .setCombiningSpec( + RunnerApi.CombiningStateSpec.newBuilder() + .setAccumulatorCoderId(registerCoderOrThrow(components, accumCoder)) + .setCombineFn(CombineTranslation.toProto(combineFn))) + .build(); + } + + @Override + public RunnerApi.StateSpec dispatchMap(Coder keyCoder, Coder valueCoder) { + return builder + .setMapSpec( + RunnerApi.MapStateSpec.newBuilder() + .setKeyCoderId(registerCoderOrThrow(components, keyCoder)) + .setValueCoderId(registerCoderOrThrow(components, valueCoder))) + .build(); + } + + @Override + public RunnerApi.StateSpec dispatchSet(Coder elementCoder) { + return builder + .setSetSpec( + RunnerApi.SetStateSpec.newBuilder() + .setElementCoderId(registerCoderOrThrow(components, elementCoder))) + .build(); + } + }); + } + + @VisibleForTesting + static StateSpec fromProto(RunnerApi.StateSpec stateSpec, RunnerApi.Components components) + throws IOException { + switch (stateSpec.getSpecCase()) { + case VALUE_SPEC: + return StateSpecs.value( + CoderTranslation.fromProto( + components.getCodersMap().get(stateSpec.getValueSpec().getCoderId()), components)); + case BAG_SPEC: + return StateSpecs.bag( + CoderTranslation.fromProto( + components.getCodersMap().get(stateSpec.getBagSpec().getElementCoderId()), + components)); + case COMBINING_SPEC: + FunctionSpec combineFnSpec = stateSpec.getCombiningSpec().getCombineFn().getSpec(); + + if (!combineFnSpec.getUrn().equals(CombineTranslation.JAVA_SERIALIZED_COMBINE_FN_URN)) { + throw new UnsupportedOperationException( + String.format( + "Cannot create %s from non-Java %s: %s", + StateSpec.class.getSimpleName(), + Combine.CombineFn.class.getSimpleName(), + combineFnSpec.getUrn())); + } + + Combine.CombineFn combineFn = + (Combine.CombineFn) + SerializableUtils.deserializeFromByteArray( + combineFnSpec.getParameter().unpack(BytesValue.class).toByteArray(), + Combine.CombineFn.class.getSimpleName()); + + // Rawtype coder cast because it is required to be a valid accumulator coder + // for the CombineFn, by construction + return StateSpecs.combining( + (Coder) + CoderTranslation.fromProto( + components + .getCodersMap() + .get(stateSpec.getCombiningSpec().getAccumulatorCoderId()), + components), + combineFn); + + case MAP_SPEC: + return StateSpecs.map( + CoderTranslation.fromProto( + components.getCodersOrThrow(stateSpec.getMapSpec().getKeyCoderId()), components), + CoderTranslation.fromProto( + components.getCodersOrThrow(stateSpec.getMapSpec().getValueCoderId()), components)); + + case SET_SPEC: + return StateSpecs.set( + CoderTranslation.fromProto( + components.getCodersMap().get(stateSpec.getSetSpec().getElementCoderId()), + components)); + + case SPEC_NOT_SET: + default: + throw new IllegalArgumentException( + String.format("Unknown %s: %s", RunnerApi.StateSpec.class.getName(), stateSpec)); + + } + } + + private static String registerCoderOrThrow(SdkComponents components, Coder coder) { + try { + return components.registerCoder(coder); + } catch (IOException exc) { + throw new RuntimeException("Failure to register coder", exc); + } } - // TODO: Implement - private static TimerSpec toProto(TimerDeclaration timer) { - throw new UnsupportedOperationException("Not yet supported"); + private static RunnerApi.TimerSpec toProto(TimerSpec timer) { + return RunnerApi.TimerSpec.newBuilder().setTimeDomain(toProto(timer.getTimeDomain())).build(); + } + + private static RunnerApi.TimeDomain toProto(TimeDomain timeDomain) { + switch(timeDomain) { + case EVENT_TIME: + return RunnerApi.TimeDomain.EVENT_TIME; + case PROCESSING_TIME: + return RunnerApi.TimeDomain.PROCESSING_TIME; + case SYNCHRONIZED_PROCESSING_TIME: + return RunnerApi.TimeDomain.SYNCHRONIZED_PROCESSING_TIME; + default: + throw new IllegalArgumentException("Unknown time domain"); + } } @AutoValue diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java index ec2795746451..46f6a806f292 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java @@ -28,6 +28,7 @@ import java.util.Map; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.Components; @@ -62,98 +63,159 @@ import org.hamcrest.Matchers; import org.junit.Test; import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; +import org.junit.runners.Suite; /** Tests for {@link ParDoTranslation}. */ -@RunWith(Parameterized.class) +@RunWith(Suite.class) +@Suite.SuiteClasses({ + ParDoTranslationTest.TestParDoPayloadTranslation.class, + ParDoTranslationTest.TestStateAndTimerTranslation.class +}) public class ParDoTranslationTest { - public static TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); - - private static PCollectionView singletonSideInput = - p.apply("GenerateSingleton", GenerateSequence.from(0L).to(1L)) - .apply(View.asSingleton()); - private static PCollectionView>> multimapSideInput = - p.apply("CreateMultimap", Create.of(KV.of(1L, "foo"), KV.of(1L, "bar"), KV.of(2L, "spam"))) - .setCoder(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of())) - .apply(View.asMultimap()); - - private static PCollection> mainInput = - p.apply("CreateMainInput", Create.empty(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of()))); - - @Parameters(name = "{index}: {0}") - public static Iterable> data() { - return ImmutableList.>of( - ParDo.of(new DropElementsFn()).withOutputTags(new TupleTag(), TupleTagList.empty()), - ParDo.of(new DropElementsFn()) - .withOutputTags(new TupleTag(), TupleTagList.empty()) - .withSideInputs(singletonSideInput, multimapSideInput), - ParDo.of(new DropElementsFn()) - .withOutputTags( - new TupleTag(), - TupleTagList.of(new TupleTag() {}).and(new TupleTag() {})) - .withSideInputs(singletonSideInput, multimapSideInput), - ParDo.of(new DropElementsFn()) - .withOutputTags( - new TupleTag(), - TupleTagList.of(new TupleTag() {}).and(new TupleTag() {}))); - } - @Parameter(0) - public ParDo.MultiOutput, Void> parDo; + /** + * Tests for translating various {@link ParDo} transforms to/from {@link ParDoPayload} protos. + */ + @RunWith(Parameterized.class) + public static class TestParDoPayloadTranslation { + public static TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + + private static PCollectionView singletonSideInput = + p.apply("GenerateSingleton", GenerateSequence.from(0L).to(1L)) + .apply(View.asSingleton()); + private static PCollectionView>> multimapSideInput = + p.apply("CreateMultimap", Create.of(KV.of(1L, "foo"), KV.of(1L, "bar"), KV.of(2L, "spam"))) + .setCoder(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of())) + .apply(View.asMultimap()); - @Test - public void testToAndFromProto() throws Exception { - SdkComponents components = SdkComponents.create(); - ParDoPayload payload = ParDoTranslation.toProto(parDo, components); + private static PCollection> mainInput = + p.apply( + "CreateMainInput", Create.empty(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of()))); - assertThat(ParDoTranslation.getDoFn(payload), Matchers.>equalTo(parDo.getFn())); - assertThat( - ParDoTranslation.getMainOutputTag(payload), - Matchers.>equalTo(parDo.getMainOutputTag())); - for (PCollectionView view : parDo.getSideInputs()) { - payload.getSideInputsOrThrow(view.getTagInternal().getId()); + @Parameters(name = "{index}: {0}") + public static Iterable> data() { + return ImmutableList.>of( + ParDo.of(new DropElementsFn()).withOutputTags(new TupleTag(), TupleTagList.empty()), + ParDo.of(new DropElementsFn()) + .withOutputTags(new TupleTag(), TupleTagList.empty()) + .withSideInputs(singletonSideInput, multimapSideInput), + ParDo.of(new DropElementsFn()) + .withOutputTags( + new TupleTag(), + TupleTagList.of(new TupleTag() {}).and(new TupleTag() {})) + .withSideInputs(singletonSideInput, multimapSideInput), + ParDo.of(new DropElementsFn()) + .withOutputTags( + new TupleTag(), + TupleTagList.of(new TupleTag() {}).and(new TupleTag() {}))); } - } - @Test - public void toAndFromTransformProto() throws Exception { - Map, PValue> inputs = new HashMap<>(); - inputs.put(new TupleTag>() {}, mainInput); - inputs.putAll(parDo.getAdditionalInputs()); - PCollectionTuple output = mainInput.apply(parDo); - - SdkComponents components = SdkComponents.create(); - String transformId = - components.registerPTransform( - AppliedPTransform.>, PCollection, MultiOutput>of( - "foo", inputs, output.expand(), parDo, p), - Collections.>emptyList()); - - Components protoComponents = components.toComponents(); - RunnerApi.PTransform protoTransform = - protoComponents.getTransformsOrThrow(transformId); - ParDoPayload parDoPayload = protoTransform.getSpec().getParameter().unpack(ParDoPayload.class); - for (PCollectionView view : parDo.getSideInputs()) { - SideInput sideInput = parDoPayload.getSideInputsOrThrow(view.getTagInternal().getId()); - PCollectionView restoredView = - ParDoTranslation.fromProto( - sideInput, view.getTagInternal().getId(), protoTransform, protoComponents); - assertThat(restoredView.getTagInternal(), equalTo(view.getTagInternal())); - assertThat(restoredView.getViewFn(), instanceOf(view.getViewFn().getClass())); + @Parameter(0) + public ParDo.MultiOutput, Void> parDo; + + @Test + public void testToAndFromProto() throws Exception { + SdkComponents components = SdkComponents.create(); + ParDoPayload payload = ParDoTranslation.toProto(parDo, components); + + assertThat(ParDoTranslation.getDoFn(payload), Matchers.>equalTo(parDo.getFn())); assertThat( - restoredView.getWindowMappingFn(), instanceOf(view.getWindowMappingFn().getClass())); + ParDoTranslation.getMainOutputTag(payload), + Matchers.>equalTo(parDo.getMainOutputTag())); + for (PCollectionView view : parDo.getSideInputs()) { + payload.getSideInputsOrThrow(view.getTagInternal().getId()); + } + } + + @Test + public void toAndFromTransformProto() throws Exception { + Map, PValue> inputs = new HashMap<>(); + inputs.put(new TupleTag>() {}, mainInput); + inputs.putAll(parDo.getAdditionalInputs()); + PCollectionTuple output = mainInput.apply(parDo); + + SdkComponents components = SdkComponents.create(); + String transformId = + components.registerPTransform( + AppliedPTransform.>, PCollection, MultiOutput>of( + "foo", inputs, output.expand(), parDo, p), + Collections.>emptyList()); + + Components protoComponents = components.toComponents(); + RunnerApi.PTransform protoTransform = protoComponents.getTransformsOrThrow(transformId); + ParDoPayload parDoPayload = + protoTransform.getSpec().getParameter().unpack(ParDoPayload.class); + for (PCollectionView view : parDo.getSideInputs()) { + SideInput sideInput = parDoPayload.getSideInputsOrThrow(view.getTagInternal().getId()); + PCollectionView restoredView = + ParDoTranslation.fromProto( + sideInput, view.getTagInternal().getId(), protoTransform, protoComponents); + assertThat(restoredView.getTagInternal(), equalTo(view.getTagInternal())); + assertThat(restoredView.getViewFn(), instanceOf(view.getViewFn().getClass())); + assertThat( + restoredView.getWindowMappingFn(), instanceOf(view.getWindowMappingFn().getClass())); + assertThat( + restoredView.getWindowingStrategyInternal(), + Matchers.>equalTo( + view.getWindowingStrategyInternal().fixDefaults())); + assertThat(restoredView.getCoderInternal(), equalTo(view.getCoderInternal())); + } + String mainInputId = components.registerPCollection(mainInput); assertThat( - restoredView.getWindowingStrategyInternal(), - Matchers.>equalTo( - view.getWindowingStrategyInternal().fixDefaults())); - assertThat(restoredView.getCoderInternal(), equalTo(view.getCoderInternal())); + ParDoTranslation.getMainInput(protoTransform, protoComponents), + equalTo(protoComponents.getPcollectionsOrThrow(mainInputId))); + } + } + + /** + * Tests for translating state and timer bits to/from protos. + */ + @RunWith(JUnit4.class) + public static class TestStateAndTimerTranslation { + + @Test + public void testValueStateSpecToFromProto() throws Exception { + SdkComponents sdkComponents = SdkComponents.create(); + StateSpec stateSpec = StateSpecs.value(VarIntCoder.of()); + StateSpec deserializedStateSpec = + ParDoTranslation.fromProto( + ParDoTranslation.toProto(stateSpec, sdkComponents), sdkComponents.toComponents()); + assertThat(stateSpec, Matchers.>equalTo(deserializedStateSpec)); + } + + @Test + public void testBagStateSpecToFromProto() throws Exception { + SdkComponents sdkComponents = SdkComponents.create(); + StateSpec stateSpec = StateSpecs.bag(VarIntCoder.of()); + StateSpec deserializedStateSpec = + ParDoTranslation.fromProto( + ParDoTranslation.toProto(stateSpec, sdkComponents), sdkComponents.toComponents()); + assertThat(stateSpec, Matchers.>equalTo(deserializedStateSpec)); + } + + @Test + public void testSetStateSpecToFromProto() throws Exception { + SdkComponents sdkComponents = SdkComponents.create(); + StateSpec stateSpec = StateSpecs.set(VarIntCoder.of()); + StateSpec deserializedStateSpec = + ParDoTranslation.fromProto( + ParDoTranslation.toProto(stateSpec, sdkComponents), sdkComponents.toComponents()); + assertThat(stateSpec, Matchers.>equalTo(deserializedStateSpec)); + } + + @Test + public void testMapStateSpecToFromProto() throws Exception { + SdkComponents sdkComponents = SdkComponents.create(); + StateSpec stateSpec = StateSpecs.map(StringUtf8Coder.of(), VarIntCoder.of()); + StateSpec deserializedStateSpec = + ParDoTranslation.fromProto( + ParDoTranslation.toProto(stateSpec, sdkComponents), sdkComponents.toComponents()); + assertThat(stateSpec, Matchers.>equalTo(deserializedStateSpec)); } - String mainInputId = components.registerPCollection(mainInput); - assertThat( - ParDoTranslation.getMainInput(protoTransform, protoComponents), - equalTo(protoComponents.getPcollectionsOrThrow(mainInputId))); } private static class DropElementsFn extends DoFn, Void> { From 247f9bc1581984d026764b3d433cb594e700bc21 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 26 May 2017 11:00:40 -0700 Subject: [PATCH 107/159] Revert "Roll-forward Include Additional PTransform inputs in Transform Nodes" This reverts commit 2e2ae9cfa581a73864695d15102acadc2750a57a. --- .../apex/translation/TranslationContext.java | 4 +- .../core/construction/TransformInputs.java | 50 ------ .../construction/TransformInputsTest.java | 166 ------------------ .../runners/direct/DirectGraphVisitor.java | 15 +- .../runners/direct/ParDoEvaluatorFactory.java | 9 +- ...ttableProcessElementsEvaluatorFactory.java | 2 - .../direct/StatefulParDoEvaluatorFactory.java | 1 - .../beam/runners/direct/WatermarkManager.java | 17 +- .../runners/direct/ParDoEvaluatorTest.java | 6 +- .../flink/FlinkBatchTranslationContext.java | 3 +- .../FlinkStreamingTranslationContext.java | 3 +- .../dataflow/DataflowPipelineTranslator.java | 5 +- .../spark/translation/EvaluationContext.java | 4 +- .../beam/sdk/runners/TransformHierarchy.java | 28 +-- 14 files changed, 29 insertions(+), 284 deletions(-) delete mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java delete mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java index 94d13e177dec..aff3863624c4 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TranslationContext.java @@ -34,7 +34,6 @@ import org.apache.beam.runners.apex.translation.utils.ApexStateInternals.ApexStateBackend; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translation.utils.CoderAdapterStreamCodec; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -94,8 +93,7 @@ public Map, PValue> getInputs() { } public InputT getInput() { - return (InputT) - Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(getCurrentTransform())); + return (InputT) Iterables.getOnlyElement(getCurrentTransform().getInputs().values()); } public Map, PValue> getOutputs() { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java deleted file mode 100644 index 2baf93a3c128..000000000000 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.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.runners.core.construction; - -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.common.collect.ImmutableList; -import java.util.Collection; -import java.util.Map; -import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.TupleTag; - -/** Utilities for extracting subsets of inputs from an {@link AppliedPTransform}. */ -public class TransformInputs { - /** - * Gets all inputs of the {@link AppliedPTransform} that are not returned by {@link - * PTransform#getAdditionalInputs()}. - */ - public static Collection nonAdditionalInputs(AppliedPTransform application) { - ImmutableList.Builder mainInputs = ImmutableList.builder(); - PTransform transform = application.getTransform(); - for (Map.Entry, PValue> input : application.getInputs().entrySet()) { - if (!transform.getAdditionalInputs().containsKey(input.getKey())) { - mainInputs.add(input.getValue()); - } - } - checkArgument( - !mainInputs.build().isEmpty() || application.getInputs().isEmpty(), - "Expected at least one main input if any inputs exist"); - return mainInputs.build(); - } -} diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java deleted file mode 100644 index f5b2c11e7923..000000000000 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.core.construction; - -import static org.junit.Assert.assertThat; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.GenerateSequence; -import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.PTransform; -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.POutput; -import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.TupleTag; -import org.hamcrest.Matchers; -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 TransformInputs}. */ -@RunWith(JUnit4.class) -public class TransformInputsTest { - @Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false); - @Rule public ExpectedException thrown = ExpectedException.none(); - - @Test - public void nonAdditionalInputsWithNoInputSucceeds() { - AppliedPTransform transform = - AppliedPTransform.of( - "input-free", - Collections., PValue>emptyMap(), - Collections., PValue>emptyMap(), - new TestTransform(), - pipeline); - - assertThat(TransformInputs.nonAdditionalInputs(transform), Matchers.empty()); - } - - @Test - public void nonAdditionalInputsWithOneMainInputSucceeds() { - PCollection input = pipeline.apply(GenerateSequence.from(1L)); - AppliedPTransform transform = - AppliedPTransform.of( - "input-single", - Collections., PValue>singletonMap(new TupleTag() {}, input), - Collections., PValue>emptyMap(), - new TestTransform(), - pipeline); - - assertThat( - TransformInputs.nonAdditionalInputs(transform), Matchers.containsInAnyOrder(input)); - } - - @Test - public void nonAdditionalInputsWithMultipleNonAdditionalInputsSucceeds() { - Map, PValue> allInputs = new HashMap<>(); - PCollection mainInts = pipeline.apply("MainInput", Create.of(12, 3)); - allInputs.put(new TupleTag() {}, mainInts); - PCollection voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of())); - allInputs.put(new TupleTag() {}, voids); - AppliedPTransform transform = - AppliedPTransform.of( - "additional-free", - allInputs, - Collections., PValue>emptyMap(), - new TestTransform(), - pipeline); - - assertThat( - TransformInputs.nonAdditionalInputs(transform), - Matchers.containsInAnyOrder(voids, mainInts)); - } - - @Test - public void nonAdditionalInputsWithAdditionalInputsSucceeds() { - Map, PValue> additionalInputs = new HashMap<>(); - additionalInputs.put(new TupleTag() {}, pipeline.apply(Create.of("1, 2", "3"))); - additionalInputs.put(new TupleTag() {}, pipeline.apply(GenerateSequence.from(3L))); - - Map, PValue> allInputs = new HashMap<>(); - PCollection mainInts = pipeline.apply("MainInput", Create.of(12, 3)); - allInputs.put(new TupleTag() {}, mainInts); - PCollection voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of())); - allInputs.put( - new TupleTag() {}, voids); - allInputs.putAll(additionalInputs); - - AppliedPTransform transform = - AppliedPTransform.of( - "additional", - allInputs, - Collections., PValue>emptyMap(), - new TestTransform(additionalInputs), - pipeline); - - assertThat( - TransformInputs.nonAdditionalInputs(transform), - Matchers.containsInAnyOrder(mainInts, voids)); - } - - @Test - public void nonAdditionalInputsWithOnlyAdditionalInputsThrows() { - Map, PValue> additionalInputs = new HashMap<>(); - additionalInputs.put(new TupleTag() {}, pipeline.apply(Create.of("1, 2", "3"))); - additionalInputs.put(new TupleTag() {}, pipeline.apply(GenerateSequence.from(3L))); - - AppliedPTransform transform = - AppliedPTransform.of( - "additional-only", - additionalInputs, - Collections., PValue>emptyMap(), - new TestTransform(additionalInputs), - pipeline); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("at least one"); - TransformInputs.nonAdditionalInputs(transform); - } - - private static class TestTransform extends PTransform { - private final Map, PValue> additionalInputs; - - private TestTransform() { - this(Collections., PValue>emptyMap()); - } - - private TestTransform(Map, PValue> additionalInputs) { - this.additionalInputs = additionalInputs; - } - - @Override - public POutput expand(PInput input) { - return PDone.in(input.getPipeline()); - } - - @Override - public Map, PValue> getAdditionalInputs() { - return additionalInputs; - } - } -} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index ed4282bbdb8b..01204e3049dd 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -21,12 +21,10 @@ import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ListMultimap; -import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -36,8 +34,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the @@ -45,7 +41,6 @@ * input after the upstream transform has produced and committed output. */ class DirectGraphVisitor extends PipelineVisitor.Defaults { - private static final Logger LOG = LoggerFactory.getLogger(DirectGraphVisitor.class); private Map> producers = new HashMap<>(); @@ -88,15 +83,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { if (node.getInputs().isEmpty()) { rootTransforms.add(appliedTransform); } else { - Collection mainInputs = - TransformInputs.nonAdditionalInputs(node.toAppliedPTransform(getPipeline())); - if (!mainInputs.containsAll(node.getInputs().values())) { - LOG.debug( - "Inputs reduced to {} from {} by removing additional inputs", - mainInputs, - node.getInputs().values()); - } - for (PValue value : mainInputs) { + for (PValue value : node.getInputs().values()) { primitiveConsumers.put(value, appliedTransform); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java index 516f798aba97..8aa75cf1445e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java @@ -20,6 +20,7 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -78,7 +79,6 @@ public TransformEvaluator forApplication( (TransformEvaluator) createEvaluator( (AppliedPTransform) application, - (PCollection) inputBundle.getPCollection(), inputBundle.getKey(), doFn, transform.getSideInputs(), @@ -102,7 +102,6 @@ public void cleanup() throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( AppliedPTransform, PCollectionTuple, ?> application, - PCollection mainInput, StructuralKey inputBundleKey, DoFn doFn, List> sideInputs, @@ -121,7 +120,6 @@ DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( createParDoEvaluator( application, inputBundleKey, - mainInput, sideInputs, mainOutputTag, additionalOutputTags, @@ -134,7 +132,6 @@ DoFnLifecycleManagerRemovingTransformEvaluator createEvaluator( ParDoEvaluator createParDoEvaluator( AppliedPTransform, PCollectionTuple, ?> application, StructuralKey key, - PCollection mainInput, List> sideInputs, TupleTag mainOutputTag, List> additionalOutputTags, @@ -147,7 +144,8 @@ ParDoEvaluator createParDoEvaluator( evaluationContext, stepContext, application, - mainInput.getWindowingStrategy(), + ((PCollection) Iterables.getOnlyElement(application.getInputs().values())) + .getWindowingStrategy(), fn, key, sideInputs, @@ -175,4 +173,5 @@ static Map, PCollection> pcollections(Map, PValue> ou } return pcs; } + } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index eccc83a031cb..b85f481c1489 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -116,8 +116,6 @@ public void cleanup() throws Exception { delegateFactory.createParDoEvaluator( application, inputBundle.getKey(), - (PCollection>>) - inputBundle.getPCollection(), transform.getSideInputs(), transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 3619d05b47ca..506c84cec639 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -117,7 +117,6 @@ private TransformEvaluator>> createEvaluator( DoFnLifecycleManagerRemovingTransformEvaluator> delegateEvaluator = delegateFactory.createEvaluator( (AppliedPTransform) application, - (PCollection) inputBundle.getPCollection(), inputBundle.getKey(), doFn, application.getTransform().getUnderlyingParDo().getSideInputs(), diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index b15b52e314de..4f1b8319dc2d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -823,11 +823,10 @@ private Collection getInputProcessingWatermarks(AppliedPTransform getInputWatermarks(AppliedPTransform transform) inputWatermarksBuilder.add(THE_END_OF_TIME); } for (PValue pvalue : inputs.values()) { - if (graph.getPrimitiveConsumers(pvalue).contains(transform)) { - Watermark producerOutputWatermark = - getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; - inputWatermarksBuilder.add(producerOutputWatermark); - } + Watermark producerOutputWatermark = + getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; + inputWatermarksBuilder.add(producerOutputWatermark); } List inputCollectionWatermarks = inputWatermarksBuilder.build(); return inputCollectionWatermarks; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index 8b86bbe50958..09a21ac524a5 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -98,7 +98,7 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { when(evaluationContext.createBundle(output)).thenReturn(outputBundle); ParDoEvaluator evaluator = - createEvaluator(singletonView, fn, inputPc, output); + createEvaluator(singletonView, fn, output); IntervalWindow nonGlobalWindow = new IntervalWindow(new Instant(0), new Instant(10_000L)); WindowedValue first = WindowedValue.valueInGlobalWindow(3); @@ -132,7 +132,6 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { private ParDoEvaluator createEvaluator( PCollectionView singletonView, RecorderFn fn, - PCollection input, PCollection output) { when( evaluationContext.createSideInputReader( @@ -157,7 +156,8 @@ private ParDoEvaluator createEvaluator( evaluationContext, stepContext, transform, - input.getWindowingStrategy(), + ((PCollection) Iterables.getOnlyElement(transform.getInputs().values())) + .getWindowingStrategy(), fn, null /* key */, ImmutableList.>of(singletonView), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java index 6e7019848b19..0439119dfc40 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java @@ -20,7 +20,6 @@ import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.Map; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -144,7 +143,7 @@ Map, PValue> getInputs(PTransform transform) { @SuppressWarnings("unchecked") T getInput(PTransform transform) { - return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform)); + return (T) Iterables.getOnlyElement(currentTransform.getInputs().values()); } Map, PValue> getOutputs(PTransform transform) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java index 74a5fb971144..ea5f6b3162af 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java @@ -22,7 +22,6 @@ import com.google.common.collect.Iterables; import java.util.HashMap; import java.util.Map; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -114,7 +113,7 @@ public TypeInformation> getTypeInfo(PCollection collecti @SuppressWarnings("unchecked") public T getInput(PTransform transform) { - return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform)); + return (T) Iterables.getOnlyElement(currentTransform.getInputs().values()); } public Map, PValue> getInputs(PTransform transform) { 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 index fccd018b13d2..af93ef59bba4 100644 --- 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 @@ -56,7 +56,6 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import javax.annotation.Nullable; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.core.construction.WindowingStrategyTranslation; import org.apache.beam.runners.dataflow.BatchViewOverrides.GroupByKeyAndSortValuesOnly; import org.apache.beam.runners.dataflow.DataflowRunner.CombineGroupedValues; @@ -396,9 +395,7 @@ public Map, PValue> getInputs( @Override public InputT getInput(PTransform transform) { - return (InputT) - Iterables.getOnlyElement( - TransformInputs.nonAdditionalInputs(getCurrentTransform(transform))); + return (InputT) Iterables.getOnlyElement(getInputs(transform).values()); } @Override diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index 0c6c4d1cb660..8102926f6daa 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -26,7 +26,6 @@ import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; -import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.sdk.Pipeline; @@ -104,8 +103,7 @@ public void setCurrentTransform(AppliedPTransform transform) { public T getInput(PTransform transform) { @SuppressWarnings("unchecked") - T input = - (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(getCurrentTransform())); + T input = (T) Iterables.getOnlyElement(getInputs(transform).values()); return input; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index bebc306a8d24..630d24cf68f1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -32,6 +32,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; @@ -67,7 +68,7 @@ public TransformHierarchy() { producers = new HashMap<>(); producerInput = new HashMap<>(); unexpandedInputs = new HashMap<>(); - root = new Node(); + root = new Node(null, null, "", null); current = root; } @@ -253,37 +254,26 @@ public class Node { @VisibleForTesting boolean finishedSpecifying = false; - /** - * Creates the root-level node. The root level node has a null enclosing node, a null transform, - * an empty map of inputs, and a name equal to the empty string. - */ - private Node() { - this.enclosingNode = null; - this.transform = null; - this.fullName = ""; - this.inputs = Collections.emptyMap(); - } - /** * Creates a new Node with the given parent and transform. * + *

EnclosingNode and transform may both be null for a root-level node, which holds all other + * nodes. + * * @param enclosingNode the composite node containing this node * @param transform the PTransform tracked by this node * @param fullName the fully qualified name of the transform * @param input the unexpanded input to the transform */ private Node( - Node enclosingNode, - PTransform transform, + @Nullable Node enclosingNode, + @Nullable PTransform transform, String fullName, - PInput input) { + @Nullable PInput input) { this.enclosingNode = enclosingNode; this.transform = transform; this.fullName = fullName; - ImmutableMap.Builder, PValue> inputs = ImmutableMap.builder(); - inputs.putAll(input.expand()); - inputs.putAll(transform.getAdditionalInputs()); - this.inputs = inputs.build(); + this.inputs = input == null ? Collections., PValue>emptyMap() : input.expand(); } /** From 6ad6433ec0c02aec8656e9e3b27f6e0f974f8ece Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 26 May 2017 11:04:05 -0700 Subject: [PATCH 108/159] Revert "Visit a Transform Hierarchy in Topological Order" This reverts commit bd1dfdf3c8e145a99bcacebd0c64dcf6580f3ffe. --- .../beam/runners/spark/SparkRunner.java | 13 --- .../spark/translation/BoundedDataset.java | 6 -- .../translation/TransformTranslator.java | 1 - .../spark/translation/StorageLevelTest.java | 4 +- .../beam/sdk/runners/TransformHierarchy.java | 46 +--------- .../sdk/runners/TransformHierarchyTest.java | 86 ------------------- 6 files changed, 6 insertions(+), 150 deletions(-) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index 1be5e13f4f5b..9e2426ef8381 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java @@ -21,10 +21,8 @@ import com.google.common.collect.Iterables; import java.util.Arrays; import java.util.Collection; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -51,7 +49,6 @@ import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PCollection; @@ -339,8 +336,6 @@ public static class Evaluator extends Pipeline.PipelineVisitor.Defaults { protected final EvaluationContext ctxt; protected final SparkPipelineTranslator translator; - private final Set shouldIgnoreChildren = new HashSet<>(); - public Evaluator(SparkPipelineTranslator translator, EvaluationContext ctxt) { this.translator = translator; this.ctxt = ctxt; @@ -356,7 +351,6 @@ public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { LOG.info("Entering directly-translatable composite transform: '{}'", node.getFullName()); LOG.debug("Composite transform class: '{}'", transformClass); doVisitTransform(node); - shouldIgnoreChildren.add(node); return CompositeBehavior.DO_NOT_ENTER_TRANSFORM; } } @@ -398,13 +392,6 @@ protected boolean shouldDefer(TransformHierarchy.Node node) { @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { - Node parent = node.getEnclosingNode(); - while (!parent.isRootNode()) { - if (shouldIgnoreChildren.contains(parent)) { - return; - } - parent = parent.getEnclosingNode(); - } doVisitTransform(node); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java index a746634008ef..652c75305f0c 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java @@ -19,7 +19,6 @@ package org.apache.beam.runners.spark.translation; import com.google.common.base.Function; -import com.google.common.base.Joiner; import com.google.common.collect.Iterables; import java.util.List; import javax.annotation.Nullable; @@ -98,13 +97,8 @@ public WindowedValue apply(byte[] bytes) { return windowedValues; } - int timesCached = 0; @Override public void cache(String storageLevel) { - System.out.printf( - "Persisting Dataset %s for RDD %s (id %s) at level %s. %s times before%n", - this, getRDD(), getRDD().toDebugString(), storageLevel, timesCached++); - System.out.println(Joiner.on("\n\t").join(new Throwable().getStackTrace())); // populate the rdd if needed getRDD().persist(StorageLevel.fromString(storageLevel)); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 6ca12c9ca6cc..742ea83539e2 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -407,7 +407,6 @@ public void evaluate(Read.Bounded transform, EvaluationContext context) { JavaRDD> input = new SourceRDD.Bounded<>( jsc.sc(), transform.getSource(), runtimeContext, stepName).toJavaRDD(); // cache to avoid re-evaluation of the source by Spark's lazy DAG evaluation. - System.out.println("Evaluating Bounded Read " + transform); context.putDataset(transform, new BoundedDataset<>(input.cache())); } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java index 8bd6dae98a74..8f2e681c9e91 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/StorageLevelTest.java @@ -58,12 +58,12 @@ public static void teardown() { @Test public void test() throws Exception { - PCollection pCollection = pipeline.apply("CreateFoo", Create.of("foo")); + PCollection pCollection = pipeline.apply(Create.of("foo")); // by default, the Spark runner doesn't cache the RDD if it accessed only one time. // So, to "force" the caching of the RDD, we have to call the RDD at least two time. // That's why we are using Count fn on the PCollection. - pCollection.apply("CountAll", Count.globally()); + pCollection.apply(Count.globally()); PCollection output = pCollection.apply(new StorageLevelPTransform()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java index 630d24cf68f1..2f0e8efd7de8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java @@ -202,12 +202,10 @@ Node getProducer(PValue produced) { return producers.get(produced); } - int traversed = 0; public Set visit(PipelineVisitor visitor) { finishSpecifying(); Set visitedValues = new HashSet<>(); - traversed++; - root.visit(visitor, visitedValues, new HashSet(), new HashSet()); + root.visit(visitor, visitedValues); return visitedValues; } @@ -464,22 +462,7 @@ public Map, PValue> getOutputs() { *

Provides an ordered visit of the input values, the primitive transform (or child nodes for * composite transforms), then the output values. */ - private void visit( - PipelineVisitor visitor, - Set visitedValues, - Set visitedNodes, - Set passedComposites) { - if (getEnclosingNode() != null && !visitedNodes.contains(getEnclosingNode())) { - getEnclosingNode().visit(visitor, visitedValues, visitedNodes, passedComposites); - } - if (!visitedNodes.add(this)) { - LOG.debug("Not revisiting previously visited node {}", this); - return; - } else if (childNodeOf(passedComposites)) { - LOG.debug("Not revisiting Node {} which is a child of a previously passed composite", this); - return; - } - + private void visit(PipelineVisitor visitor, Set visitedValues) { if (!finishedSpecifying) { finishSpecifying(); } @@ -487,31 +470,22 @@ private void visit( if (!isRootNode()) { // Visit inputs. for (PValue inputValue : inputs.values()) { - Node valueProducer = getProducer(inputValue); - if (!visitedNodes.contains(valueProducer)) { - valueProducer.visit(visitor, visitedValues, visitedNodes, passedComposites); - } if (visitedValues.add(inputValue)) { - LOG.debug("Visiting input value {}", inputValue); - visitor.visitValue(inputValue, valueProducer); + visitor.visitValue(inputValue, getProducer(inputValue)); } } } if (isCompositeNode()) { - LOG.debug("Visiting composite node {}", this); PipelineVisitor.CompositeBehavior recurse = visitor.enterCompositeTransform(this); if (recurse.equals(CompositeBehavior.ENTER_TRANSFORM)) { for (Node child : parts) { - child.visit(visitor, visitedValues, visitedNodes, passedComposites); + child.visit(visitor, visitedValues); } - } else { - passedComposites.add(this); } visitor.leaveCompositeTransform(this); } else { - LOG.debug("Visiting primitive node {}", this); visitor.visitPrimitiveTransform(this); } @@ -520,24 +494,12 @@ private void visit( // Visit outputs. for (PValue pValue : outputs.values()) { if (visitedValues.add(pValue)) { - LOG.debug("Visiting output value {}", pValue); visitor.visitValue(pValue, this); } } } } - private boolean childNodeOf(Set nodes) { - if (isRootNode()) { - return false; - } - Node parent = this.getEnclosingNode(); - while (!parent.isRootNode() && !nodes.contains(parent)) { - parent = parent.getEnclosingNode(); - } - return nodes.contains(parent); - } - /** * Finish specifying a transform. * diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java index 2fe2817f534c..1197d1b04eb6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java @@ -19,7 +19,6 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; @@ -33,8 +32,6 @@ import java.util.Set; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.Pipeline.PipelineVisitor.Defaults; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.Read; @@ -495,87 +492,4 @@ public void visitPrimitiveTransform(Node node) { assertThat(visitedPrimitiveNodes, containsInAnyOrder(upstreamNode, replacementParNode)); assertThat(visitedValues, Matchers.containsInAnyOrder(upstream, output)); } - - @Test - public void visitIsTopologicallyOrdered() { - PCollection one = - PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED) - .setCoder(StringUtf8Coder.of()); - final PCollection two = - PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) - .setCoder(VarIntCoder.of()); - final PDone done = PDone.in(pipeline); - final TupleTag oneTag = new TupleTag() {}; - final TupleTag twoTag = new TupleTag() {}; - final PCollectionTuple oneAndTwo = PCollectionTuple.of(oneTag, one).and(twoTag, two); - - hierarchy.pushNode("consumes_both", one, new PTransform, PDone>() { - @Override - public PDone expand(PCollection input) { - return done; - } - - @Override - public Map, PValue> getAdditionalInputs() { - return Collections., PValue>singletonMap(twoTag, two); - } - }); - hierarchy.setOutput(done); - hierarchy.popNode(); - - final PTransform producer = - new PTransform() { - @Override - public PCollectionTuple expand(PBegin input) { - return oneAndTwo; - } - }; - hierarchy.pushNode( - "encloses_producer", - PBegin.in(pipeline), - new PTransform() { - @Override - public PCollectionTuple expand(PBegin input) { - return input.apply(producer); - } - }); - hierarchy.pushNode( - "creates_one_and_two", - PBegin.in(pipeline), producer); - hierarchy.setOutput(oneAndTwo); - hierarchy.popNode(); - hierarchy.setOutput(oneAndTwo); - hierarchy.popNode(); - - hierarchy.visit(new PipelineVisitor.Defaults() { - private final Set visitedNodes = new HashSet<>(); - private final Set visitedValues = new HashSet<>(); - @Override - public CompositeBehavior enterCompositeTransform(Node node) { - for (PValue input : node.getInputs().values()) { - assertThat(visitedValues, hasItem(input)); - } - visitedNodes.add(node); - return CompositeBehavior.ENTER_TRANSFORM; - } - - @Override - public void visitPrimitiveTransform(Node node) { - assertThat(visitedNodes, hasItem(node.getEnclosingNode())); - for (PValue input : node.getInputs().values()) { - assertThat(visitedValues, hasItem(input)); - } - visitedNodes.add(node); - } - - @Override - public void visitValue(PValue value, Node producer) { - assertThat(visitedNodes, hasItem(producer)); - assertThat(visitedValues, not(hasItem(value))); - visitedValues.add(value); - } - }); - } } From a251d01ab26aeada45a7c3824af295071c4d7df4 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Fri, 26 May 2017 13:46:12 -0700 Subject: [PATCH 109/159] [BEAM-1542] SpannerIO sink updates * Add unit and integration tests * Add MutationSizeEstimator * Improve display data * Upgrade libraries --- pom.xml | 42 +-- sdks/java/io/google-cloud-platform/pom.xml | 20 +- .../io/gcp/spanner/MutationSizeEstimator.java | 98 +++++++ .../beam/sdk/io/gcp/spanner/SpannerIO.java | 182 +++++++++---- .../spanner/MutationSizeEstimatorTest.java | 138 ++++++++++ .../sdk/io/gcp/spanner/SpannerIOTest.java | 244 ++++++++++++++++++ .../sdk/io/gcp/spanner/SpannerWriteIT.java | 159 ++++++++++++ 7 files changed, 808 insertions(+), 75 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java diff --git a/pom.xml b/pom.xml index 7e06b5f12952..ba1cf37d0372 100644 --- a/pom.xml +++ b/pom.xml @@ -125,7 +125,7 @@ 0.5.160304 20.0 1.2.0 - 0.1.0 + 0.1.9 1.3 2.8.8 3.0.1 @@ -176,8 +176,8 @@ - release @@ -494,7 +494,7 @@ beam-sdks-java-io-hadoop-input-format ${project.version} - + org.apache.beam beam-runners-core-construction-java @@ -738,13 +738,13 @@ google-auth-library-credentials ${google-auth.version} - + com.google.auth google-auth-library-oauth2-http ${google-auth.version} - com.google.guava @@ -809,12 +809,24 @@ + + com.google.api.grpc + proto-google-cloud-spanner-admin-database-v1 + ${grpc-google-common-protos.version} + + + + com.google.api.grpc + proto-google-common-protos + ${grpc-google-common-protos.version} + + com.google.apis google-api-services-storage ${storage.version} - com.google.guava @@ -901,7 +913,7 @@ google-api-services-dataflow ${dataflow.version} - com.google.guava @@ -915,7 +927,7 @@ google-api-services-clouddebugger ${clouddebugger.version} - com.google.guava @@ -1016,7 +1028,7 @@ byte-buddy 1.6.8 - + org.springframework spring-expression @@ -1123,7 +1135,7 @@ maven-antrun-plugin 1.8 - + org.apache.maven.plugins maven-checkstyle-plugin @@ -1394,7 +1406,7 @@ - org.eclipse.m2e @@ -1731,7 +1743,7 @@ ${basedir}/sdks/python - + ${basedir} @@ -1740,8 +1752,8 @@ README.md - - + + diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 9143ccf553cf..8b5382092ec6 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -86,11 +86,6 @@ grpc-core - - com.google.api.grpc - grpc-google-common-protos - - com.google.apis google-api-services-bigquery @@ -248,6 +243,21 @@ true + + com.google.api.grpc + proto-google-cloud-spanner-admin-database-v1 + + + + com.google.api.grpc + proto-google-common-protos + + + + org.apache.commons + commons-lang3 + + org.apache.beam diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java new file mode 100644 index 000000000000..61652e736e90 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimator.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.gcp.spanner; + +import com.google.cloud.ByteArray; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Value; + +/** Estimates the logical size of {@link com.google.cloud.spanner.Mutation}. */ +class MutationSizeEstimator { + + // Prevent construction. + private MutationSizeEstimator() {} + + /** Estimates a size of mutation in bytes. */ + static long sizeOf(Mutation m) { + long result = 0; + for (Value v : m.getValues()) { + switch (v.getType().getCode()) { + case ARRAY: + result += estimateArrayValue(v); + break; + case STRUCT: + throw new IllegalArgumentException("Structs are not supported in mutation."); + default: + result += estimatePrimitiveValue(v); + } + } + return result; + } + + private static long estimatePrimitiveValue(Value v) { + switch (v.getType().getCode()) { + case BOOL: + return 1; + case INT64: + case FLOAT64: + return 8; + case DATE: + case TIMESTAMP: + return 12; + case STRING: + return v.isNull() ? 0 : v.getString().length(); + case BYTES: + return v.isNull() ? 0 : v.getBytes().length(); + } + throw new IllegalArgumentException("Unsupported type " + v.getType()); + } + + private static long estimateArrayValue(Value v) { + switch (v.getType().getArrayElementType().getCode()) { + case BOOL: + return v.getBoolArray().size(); + case INT64: + return 8 * v.getInt64Array().size(); + case FLOAT64: + return 8 * v.getFloat64Array().size(); + case STRING: + long totalLength = 0; + for (String s : v.getStringArray()) { + if (s == null) { + continue; + } + totalLength += s.length(); + } + return totalLength; + case BYTES: + totalLength = 0; + for (ByteArray bytes : v.getBytesArray()) { + if (bytes == null) { + continue; + } + totalLength += bytes.length(); + } + return totalLength; + case DATE: + return 12 * v.getDateArray().size(); + case TIMESTAMP: + return 12 * v.getTimestampArray().size(); + } + throw new IllegalArgumentException("Unsupported type " + v.getType()); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index ec119311c106..c5325bb5a85f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -20,6 +20,8 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.google.auto.value.AutoValue; +import com.google.cloud.ServiceFactory; +import com.google.cloud.ServiceOptions; import com.google.cloud.spanner.AbortedException; import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.DatabaseId; @@ -32,6 +34,7 @@ import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -48,8 +51,8 @@ import org.slf4j.LoggerFactory; /** - * Experimental {@link PTransform Transforms} for reading from and writing to - * Google Cloud Spanner. + * Experimental {@link PTransform Transforms} for reading from and writing to Google Cloud Spanner. * *

Reading from Cloud Spanner

* @@ -72,21 +75,35 @@ * mutations.apply( * "Write", SpannerIO.write().withInstanceId("instance").withDatabaseId("database")); * }
+ * + *

The default size of the batch is set to 1MB, to override this use {@link + * Write#withBatchSizeBytes(long)}. Setting batch size to a small value or zero practically disables + * batching. + * + *

The transform does not provide same transactional guarantees as Cloud Spanner. In particular, + * + *

    + *
  • Mutations are not submitted atomically; + *
  • A mutation is applied at least once; + *
  • If the pipeline was unexpectedly stopped, mutations that were already applied will not get + * rolled back. + *
*/ @Experimental(Experimental.Kind.SOURCE_SINK) public class SpannerIO { - @VisibleForTesting - static final int SPANNER_MUTATIONS_PER_COMMIT_LIMIT = 20000; + private static final long DEFAULT_BATCH_SIZE_BYTES = 1024 * 1024; // 1 MB /** - * Creates an unitialized instance of {@link Write}. Before use, the {@link Write} must be + * Creates an uninitialized instance of {@link Write}. Before use, the {@link Write} must be * configured with a {@link Write#withInstanceId} and {@link Write#withDatabaseId} that identify * the Cloud Spanner database being written. */ @Experimental public static Write write() { - return new AutoValue_SpannerIO_Write.Builder().build(); + return new AutoValue_SpannerIO_Write.Builder() + .setBatchSizeBytes(DEFAULT_BATCH_SIZE_BYTES) + .build(); } /** @@ -98,24 +115,57 @@ public static Write write() { @AutoValue public abstract static class Write extends PTransform, PDone> { + @Nullable + abstract String getProjectId(); + @Nullable abstract String getInstanceId(); @Nullable abstract String getDatabaseId(); + abstract long getBatchSizeBytes(); + + @Nullable + @VisibleForTesting + abstract ServiceFactory getServiceFactory(); + abstract Builder toBuilder(); @AutoValue.Builder abstract static class Builder { + abstract Builder setProjectId(String projectId); + abstract Builder setInstanceId(String instanceId); abstract Builder setDatabaseId(String databaseId); + abstract Builder setBatchSizeBytes(long batchSizeBytes); + + @VisibleForTesting + abstract Builder setServiceFactory(ServiceFactory serviceFactory); + abstract Write build(); } + SpannerOptions getSpannerOptions() { + SpannerOptions.Builder builder = SpannerOptions.newBuilder(); + if (getServiceFactory() != null) { + builder.setServiceFactory(getServiceFactory()); + } + return builder.build(); + } + + /** + * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner project. + * + *

Does not modify this object. + */ + public Write withProjectId(String projectId) { + return toBuilder().setProjectId(projectId).build(); + } + /** * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner * instance. @@ -126,6 +176,15 @@ public Write withInstanceId(String instanceId) { return toBuilder().setInstanceId(instanceId).build(); } + /** + * Returns a new {@link SpannerIO.Write} with a new batch size limit. + * + *

Does not modify this object. + */ + public Write withBatchSizeBytes(long batchSizeBytes) { + return toBuilder().setBatchSizeBytes(batchSizeBytes).build(); + } + /** * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner * database. @@ -136,12 +195,24 @@ public Write withDatabaseId(String databaseId) { return toBuilder().setDatabaseId(databaseId).build(); } + @VisibleForTesting + Write withServiceFactory(ServiceFactory serviceFactory) { + return toBuilder().setServiceFactory(serviceFactory).build(); + } + @Override - public PDone expand(PCollection input) { - input.apply("Write mutations to Spanner", - ParDo.of(new SpannerWriterFn( - getInstanceId(), getDatabaseId(), SPANNER_MUTATIONS_PER_COMMIT_LIMIT))); + public void validate(PipelineOptions options) { + checkNotNull( + getInstanceId(), + "SpannerIO.write() requires instance id to be set with withInstanceId method"); + checkNotNull( + getDatabaseId(), + "SpannerIO.write() requires database id to be set with withDatabaseId method"); + } + @Override + public PDone expand(PCollection input) { + input.apply("Write mutations to Cloud Spanner", ParDo.of(new SpannerWriteFn(this))); return PDone.in(input.getPipeline()); } @@ -149,64 +220,69 @@ public PDone expand(PCollection input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull(DisplayData.item("instanceId", getInstanceId()) - .withLabel("Output Instance")) - .addIfNotNull(DisplayData.item("databaseId", getDatabaseId()) - .withLabel("Output Database")); + .addIfNotNull(DisplayData.item("projectId", getProjectId()).withLabel("Output Project")) + .addIfNotNull( + DisplayData.item("instanceId", getInstanceId()).withLabel("Output Instance")) + .addIfNotNull( + DisplayData.item("databaseId", getDatabaseId()).withLabel("Output Database")) + .add(DisplayData.item("batchSizeBytes", getBatchSizeBytes()) + .withLabel("Batch Size in Bytes")); + if (getServiceFactory() != null) { + builder.addIfNotNull( + DisplayData.item("serviceFactory", getServiceFactory().getClass().getName()) + .withLabel("Service Factory")); + } } } - /** - * {@link DoFn} that writes {@link Mutation}s to Google Cloud Spanner. Mutations are written in - * batches, where the maximum batch size is {@link SpannerIO#SPANNER_MUTATIONS_PER_COMMIT_LIMIT}. - * - *

Commits are non-transactional. If a commit fails, it will be retried (up to - * {@link SpannerWriterFn#MAX_RETRIES} times). This means that the mutation operation should be - * idempotent. - * - *

See Google Cloud Spanner documentation. - */ + /** Batches together and writes mutations to Google Cloud Spanner. */ @VisibleForTesting - static class SpannerWriterFn extends DoFn { - private static final Logger LOG = LoggerFactory.getLogger(SpannerWriterFn.class); + static class SpannerWriteFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(SpannerWriteFn.class); + private final Write spec; private transient Spanner spanner; - private final String instanceId; - private final String databaseId; - private final int batchSize; private transient DatabaseClient dbClient; // Current batch of mutations to be written. - private final List mutations = new ArrayList<>(); + private List mutations; + private long batchSizeBytes = 0; private static final int MAX_RETRIES = 5; private static final FluentBackoff BUNDLE_WRITE_BACKOFF = FluentBackoff.DEFAULT - .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); + .withMaxRetries(MAX_RETRIES) + .withInitialBackoff(Duration.standardSeconds(5)); @VisibleForTesting - SpannerWriterFn(String instanceId, String databaseId, int batchSize) { - this.instanceId = checkNotNull(instanceId, "instanceId"); - this.databaseId = checkNotNull(databaseId, "databaseId"); - this.batchSize = batchSize; + SpannerWriteFn(Write spec) { + this.spec = spec; } @Setup public void setup() throws Exception { - SpannerOptions options = SpannerOptions.newBuilder().build(); - spanner = options.getService(); - dbClient = spanner.getDatabaseClient( - DatabaseId.of(options.getProjectId(), instanceId, databaseId)); + spanner = spec.getSpannerOptions().getService(); + dbClient = + spanner.getDatabaseClient( + DatabaseId.of(projectId(), spec.getInstanceId(), spec.getDatabaseId())); + mutations = new ArrayList<>(); + batchSizeBytes = 0; } @ProcessElement public void processElement(ProcessContext c) throws Exception { Mutation m = c.element(); mutations.add(m); - int columnCount = m.asMap().size(); - if ((mutations.size() + 1) * columnCount >= batchSize) { + batchSizeBytes += MutationSizeEstimator.sizeOf(m); + if (batchSizeBytes >= spec.getBatchSizeBytes()) { flushBatch(); } } + private String projectId() { + return spec.getProjectId() == null + ? ServiceOptions.getDefaultProjectId() + : spec.getProjectId(); + } + @FinishBundle public void finishBundle() throws Exception { if (!mutations.isEmpty()) { @@ -217,20 +293,20 @@ public void finishBundle() throws Exception { @Teardown public void teardown() throws Exception { if (spanner == null) { - return; + return; } spanner.closeAsync().get(); + spanner = null; } /** * Writes a batch of mutations to Cloud Spanner. * - *

If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. - * If the retry limit is exceeded, the last exception from Cloud Spanner will be - * thrown. + *

If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. If the retry limit + * is exceeded, the last exception from Cloud Spanner will be thrown. * * @throws AbortedException if the commit fails or IOException or InterruptedException if - * backing off between retries fails. + * backing off between retries fails. */ private void flushBatch() throws AbortedException, IOException, InterruptedException { LOG.debug("Writing batch of {} mutations", mutations.size()); @@ -247,8 +323,8 @@ private void flushBatch() throws AbortedException, IOException, InterruptedExcep } catch (AbortedException exception) { // Only log the code and message for potentially-transient errors. The entire exception // will be propagated upon the last retry. - LOG.error("Error writing to Spanner ({}): {}", exception.getCode(), - exception.getMessage()); + LOG.error( + "Error writing to Spanner ({}): {}", exception.getCode(), exception.getMessage()); if (!BackOffUtils.next(sleeper, backoff)) { LOG.error("Aborting after {} retries.", MAX_RETRIES); throw exception; @@ -256,20 +332,16 @@ private void flushBatch() throws AbortedException, IOException, InterruptedExcep } } LOG.debug("Successfully wrote {} mutations", mutations.size()); - mutations.clear(); + mutations = new ArrayList<>(); + batchSizeBytes = 0; } @Override public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); - builder - .addIfNotNull(DisplayData.item("instanceId", instanceId) - .withLabel("Instance")) - .addIfNotNull(DisplayData.item("databaseId", databaseId) - .withLabel("Database")); + spec.populateDisplayData(builder); } } private SpannerIO() {} // Prevent construction. - } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java new file mode 100644 index 000000000000..03eb28ed943d --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.gcp.spanner; + +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +import com.google.cloud.ByteArray; +import com.google.cloud.Date; +import com.google.cloud.Timestamp; +import com.google.cloud.spanner.Mutation; +import java.util.Arrays; +import org.junit.Test; + +/** A set of unit tests for {@link MutationSizeEstimator}. */ +public class MutationSizeEstimatorTest { + + @Test + public void primitives() throws Exception { + Mutation int64 = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation float64 = Mutation.newInsertOrUpdateBuilder("test").set("one").to(2.9).build(); + Mutation bool = Mutation.newInsertOrUpdateBuilder("test").set("one").to(false).build(); + + assertThat(MutationSizeEstimator.sizeOf(int64), is(8L)); + assertThat(MutationSizeEstimator.sizeOf(float64), is(8L)); + assertThat(MutationSizeEstimator.sizeOf(bool), is(1L)); + } + + @Test + public void primitiveArrays() throws Exception { + Mutation int64 = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toInt64Array(new long[] {1L, 2L, 3L}) + .build(); + Mutation float64 = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toFloat64Array(new double[] {1., 2.}) + .build(); + Mutation bool = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toBoolArray(new boolean[] {true, true, false, true}) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(int64), is(24L)); + assertThat(MutationSizeEstimator.sizeOf(float64), is(16L)); + assertThat(MutationSizeEstimator.sizeOf(bool), is(4L)); + } + + @Test + public void strings() throws Exception { + Mutation emptyString = Mutation.newInsertOrUpdateBuilder("test").set("one").to("").build(); + Mutation nullString = + Mutation.newInsertOrUpdateBuilder("test").set("one").to((String) null).build(); + Mutation sampleString = Mutation.newInsertOrUpdateBuilder("test").set("one").to("abc").build(); + Mutation sampleArray = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toStringArray(Arrays.asList("one", "two", null)) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(emptyString), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(nullString), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(sampleString), is(3L)); + assertThat(MutationSizeEstimator.sizeOf(sampleArray), is(6L)); + } + + @Test + public void bytes() throws Exception { + Mutation empty = + Mutation.newInsertOrUpdateBuilder("test").set("one").to(ByteArray.fromBase64("")).build(); + Mutation nullValue = + Mutation.newInsertOrUpdateBuilder("test").set("one").to((ByteArray) null).build(); + Mutation sample = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .to(ByteArray.fromBase64("abcdabcd")) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(empty), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(nullValue), is(0L)); + assertThat(MutationSizeEstimator.sizeOf(sample), is(6L)); + } + + @Test + public void dates() throws Exception { + Mutation timestamp = + Mutation.newInsertOrUpdateBuilder("test").set("one").to(Timestamp.now()).build(); + Mutation nullTimestamp = + Mutation.newInsertOrUpdateBuilder("test").set("one").to((Timestamp) null).build(); + Mutation date = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .to(Date.fromYearMonthDay(2017, 10, 10)) + .build(); + Mutation nullDate = + Mutation.newInsertOrUpdateBuilder("test").set("one").to((Date) null).build(); + Mutation timestampArray = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toTimestampArray(Arrays.asList(Timestamp.now(), null)) + .build(); + Mutation dateArray = + Mutation.newInsertOrUpdateBuilder("test") + .set("one") + .toDateArray( + Arrays.asList( + null, + Date.fromYearMonthDay(2017, 1, 1), + null, + Date.fromYearMonthDay(2017, 1, 2))) + .build(); + + assertThat(MutationSizeEstimator.sizeOf(timestamp), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(date), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(nullTimestamp), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(nullDate), is(12L)); + assertThat(MutationSizeEstimator.sizeOf(timestampArray), is(24L)); + assertThat(MutationSizeEstimator.sizeOf(dateArray), is(48L)); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java new file mode 100644 index 000000000000..5bdfea5522b2 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOTest.java @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.gcp.spanner; + +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; + +import com.google.api.core.ApiFuture; +import com.google.cloud.ServiceFactory; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerOptions; +import com.google.common.collect.Iterables; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import javax.annotation.concurrent.GuardedBy; + +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFnTester; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentMatcher; +import org.mockito.Matchers; + + +/** + * Unit tests for {@link SpannerIO}. + */ +@RunWith(JUnit4.class) +public class SpannerIOTest implements Serializable { + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); + + private FakeServiceFactory serviceFactory; + + @Before + @SuppressWarnings("unchecked") + public void setUp() throws Exception { + serviceFactory = new FakeServiceFactory(); + } + + @Test + public void emptyTransform() throws Exception { + SpannerIO.Write write = SpannerIO.write(); + thrown.expect(NullPointerException.class); + thrown.expectMessage("requires instance id to be set with"); + write.validate(null); + } + + @Test + public void emptyInstanceId() throws Exception { + SpannerIO.Write write = SpannerIO.write().withDatabaseId("123"); + thrown.expect(NullPointerException.class); + thrown.expectMessage("requires instance id to be set with"); + write.validate(null); + } + + @Test + public void emptyDatabaseId() throws Exception { + SpannerIO.Write write = SpannerIO.write().withInstanceId("123"); + thrown.expect(NullPointerException.class); + thrown.expectMessage("requires database id to be set with"); + write.validate(null); + } + + @Test + @Category(NeedsRunner.class) + public void singleMutationPipeline() throws Exception { + Mutation mutation = Mutation.newInsertOrUpdateBuilder("test").set("one").to(2).build(); + PCollection mutations = pipeline.apply(Create.of(mutation)); + + mutations.apply( + SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withServiceFactory(serviceFactory)); + pipeline.run(); + verify(serviceFactory.mockSpanner()) + .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); + verify(serviceFactory.mockDatabaseClient(), times(1)) + .writeAtLeastOnce(argThat(new IterableOfSize(1))); + } + + @Test + public void batching() throws Exception { + Mutation one = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation two = Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build(); + SpannerIO.Write write = + SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withBatchSizeBytes(1000000000) + .withServiceFactory(serviceFactory); + SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); + DoFnTester fnTester = DoFnTester.of(writerFn); + fnTester.processBundle(Arrays.asList(one, two)); + + verify(serviceFactory.mockSpanner()) + .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); + verify(serviceFactory.mockDatabaseClient(), times(1)) + .writeAtLeastOnce(argThat(new IterableOfSize(2))); + } + + @Test + public void batchingGroups() throws Exception { + Mutation one = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation two = Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build(); + Mutation three = Mutation.newInsertOrUpdateBuilder("test").set("three").to(3).build(); + + // Have a room to accumulate one more item. + long batchSize = MutationSizeEstimator.sizeOf(one) + 1; + + SpannerIO.Write write = + SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withBatchSizeBytes(batchSize) + .withServiceFactory(serviceFactory); + SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); + DoFnTester fnTester = DoFnTester.of(writerFn); + fnTester.processBundle(Arrays.asList(one, two, three)); + + verify(serviceFactory.mockSpanner()) + .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); + verify(serviceFactory.mockDatabaseClient(), times(1)) + .writeAtLeastOnce(argThat(new IterableOfSize(2))); + verify(serviceFactory.mockDatabaseClient(), times(1)) + .writeAtLeastOnce(argThat(new IterableOfSize(1))); + } + + @Test + public void noBatching() throws Exception { + Mutation one = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build(); + Mutation two = Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build(); + SpannerIO.Write write = + SpannerIO.write() + .withProjectId("test-project") + .withInstanceId("test-instance") + .withDatabaseId("test-database") + .withBatchSizeBytes(0) // turn off batching. + .withServiceFactory(serviceFactory); + SpannerIO.SpannerWriteFn writerFn = new SpannerIO.SpannerWriteFn(write); + DoFnTester fnTester = DoFnTester.of(writerFn); + fnTester.processBundle(Arrays.asList(one, two)); + + verify(serviceFactory.mockSpanner()) + .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database")); + verify(serviceFactory.mockDatabaseClient(), times(2)) + .writeAtLeastOnce(argThat(new IterableOfSize(1))); + } + + private static class FakeServiceFactory + implements ServiceFactory, Serializable { + // Marked as static so they could be returned by serviceFactory, which is serializable. + private static final Object lock = new Object(); + + @GuardedBy("lock") + private static final List mockSpanners = new ArrayList<>(); + + @GuardedBy("lock") + private static final List mockDatabaseClients = new ArrayList<>(); + + @GuardedBy("lock") + private static int count = 0; + + private final int index; + + public FakeServiceFactory() { + synchronized (lock) { + index = count++; + mockSpanners.add(mock(Spanner.class, withSettings().serializable())); + mockDatabaseClients.add(mock(DatabaseClient.class, withSettings().serializable())); + } + ApiFuture voidFuture = mock(ApiFuture.class, withSettings().serializable()); + when(mockSpanner().getDatabaseClient(Matchers.any(DatabaseId.class))) + .thenReturn(mockDatabaseClient()); + when(mockSpanner().closeAsync()).thenReturn(voidFuture); + } + + DatabaseClient mockDatabaseClient() { + synchronized (lock) { + return mockDatabaseClients.get(index); + } + } + + Spanner mockSpanner() { + synchronized (lock) { + return mockSpanners.get(index); + } + } + + @Override + public Spanner create(SpannerOptions serviceOptions) { + return mockSpanner(); + } + } + + private static class IterableOfSize extends ArgumentMatcher> { + private final int size; + + private IterableOfSize(int size) { + this.size = size; + } + + @Override + public boolean matches(Object argument) { + return argument instanceof Iterable && Iterables.size((Iterable) argument) == size; + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java new file mode 100644 index 000000000000..064c65eedcef --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.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.sdk.io.gcp.spanner; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +import com.google.cloud.spanner.Database; +import com.google.cloud.spanner.DatabaseAdminClient; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.DatabaseId; +import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Operation; +import com.google.cloud.spanner.ResultSet; +import com.google.cloud.spanner.Spanner; +import com.google.cloud.spanner.SpannerOptions; +import com.google.cloud.spanner.Statement; +import com.google.spanner.admin.database.v1.CreateDatabaseMetadata; +import java.util.Collections; +import org.apache.beam.sdk.io.GenerateSequence; +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.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** End-to-end test of Cloud Spanner Sink. */ +@RunWith(JUnit4.class) +public class SpannerWriteIT { + @Rule public final transient TestPipeline p = TestPipeline.create(); + + /** Pipeline options for this test. */ + public interface SpannerTestPipelineOptions extends TestPipelineOptions { + @Description("Project ID for Spanner") + @Default.String("apache-beam-testing") + String getProjectId(); + void setProjectId(String value); + + @Description("Instance ID to write to in Spanner") + @Default.String("beam-test") + String getInstanceId(); + void setInstanceId(String value); + + @Description("Database ID to write to in Spanner") + @Default.String("beam-testdb") + String getDatabaseId(); + void setDatabaseId(String value); + + @Description("Table name") + @Default.String("users") + String getTable(); + void setTable(String value); + } + + private Spanner spanner; + private DatabaseAdminClient databaseAdminClient; + private SpannerTestPipelineOptions options; + + @Before + public void setUp() throws Exception { + PipelineOptionsFactory.register(SpannerTestPipelineOptions.class); + options = TestPipeline.testingPipelineOptions().as(SpannerTestPipelineOptions.class); + + spanner = SpannerOptions.newBuilder().setProjectId(options.getProjectId()).build().getService(); + + databaseAdminClient = spanner.getDatabaseAdminClient(); + + // Delete database if exists. + databaseAdminClient.dropDatabase(options.getInstanceId(), options.getDatabaseId()); + + Operation op = + databaseAdminClient.createDatabase( + options.getInstanceId(), + options.getDatabaseId(), + Collections.singleton( + "CREATE TABLE " + + options.getTable() + + " (" + + " Key INT64," + + " Value STRING(MAX)," + + ") PRIMARY KEY (Key)")); + op.waitFor(); + } + + @Test + public void testWrite() throws Exception { + p.apply(GenerateSequence.from(0).to(100)) + .apply(ParDo.of(new GenerateMutations(options.getTable()))) + .apply( + SpannerIO.write() + .withProjectId(options.getProjectId()) + .withInstanceId(options.getInstanceId()) + .withDatabaseId(options.getDatabaseId())); + + p.run(); + DatabaseClient databaseClient = + spanner.getDatabaseClient( + DatabaseId.of( + options.getProjectId(), options.getInstanceId(), options.getDatabaseId())); + + ResultSet resultSet = + databaseClient + .singleUse() + .executeQuery(Statement.of("SELECT COUNT(*) FROM " + options.getTable())); + assertThat(resultSet.next(), is(true)); + assertThat(resultSet.getLong(0), equalTo(100L)); + assertThat(resultSet.next(), is(false)); + } + + @After + public void tearDown() throws Exception { + databaseAdminClient.dropDatabase(options.getInstanceId(), options.getDatabaseId()); + spanner.closeAsync().get(); + } + + private static class GenerateMutations extends DoFn { + private final String table; + private final int valueSize = 100; + + public GenerateMutations(String table) { + this.table = table; + } + + @ProcessElement + public void processElement(ProcessContext c) { + Mutation.WriteBuilder builder = Mutation.newInsertOrUpdateBuilder(table); + Long key = c.element(); + builder.set("Key").to(key); + builder.set("Value").to(RandomStringUtils.random(valueSize, true, true)); + Mutation mutation = builder.build(); + c.output(mutation); + } + } +} From 6ec64b90b66de8693c833145c583ecb9cb2ae0f5 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 25 May 2017 16:17:54 -0700 Subject: [PATCH 110/159] [BEAM-2366] Don't try to pickle gen_protos in tests. Tests that pickle the main session but are run from setup.py attempt to pickle everything visible in setup.py. --- sdks/python/setup.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 8a8ce482742a..596c8c51afb0 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -36,8 +36,6 @@ from pkg_resources import get_distribution, DistributionNotFound -import gen_protos - def get_version(): global_names = {} @@ -127,6 +125,9 @@ def get_version(): # We must generate protos after setup_requires are installed. def generate_protos_first(original_cmd): + # See https://issues.apache.org/jira/browse/BEAM-2366 + # pylint: disable=wrong-import-position + import gen_protos class cmd(original_cmd, object): def run(self): gen_protos.generate_proto_files() From ba142aa82713a24f3d0d5b93cc13b957d1192a99 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 26 May 2017 15:13:20 -0700 Subject: [PATCH 111/159] Add Missing Space in DataflowMetrics --- .../java/org/apache/beam/runners/dataflow/DataflowMetrics.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java index a6a6a43e3f65..f038e3f8fc6e 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java @@ -149,7 +149,7 @@ private MetricQueryResults populateMetricQueryResults( || metricName.endsWith("[MEAN]") || metricName.endsWith("[COUNT]")) { // Skip distribution metrics, as these are not yet properly supported. LOG.warn("Distribution metrics are not yet supported. You can see them in the Dataflow" - + "User Interface"); + + " User Interface"); continue; } From 3878556cdf575e11fb47d0b9c1550cdfc2b60131 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 26 May 2017 11:40:38 -0700 Subject: [PATCH 112/159] [BEAM-2372] Only run Apache RAT at root pom.xml Since it checks the entire project anyway. --- pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index ba1cf37d0372..398cda09d9d3 100644 --- a/pom.xml +++ b/pom.xml @@ -243,6 +243,8 @@ org.apache.rat apache-rat-plugin + + false From 4d86cece9b27c7db13bb7c3ae3f0249e16273ac2 Mon Sep 17 00:00:00 2001 From: Michael Luckey Date: Fri, 26 May 2017 15:52:42 +0200 Subject: [PATCH 113/159] [BEAM-2369] HadoopFileSystem: prevent NPE on match of non existing file --- .../beam/sdk/io/hdfs/HadoopFileSystem.java | 7 ++++ .../sdk/io/hdfs/HadoopFileSystemTest.java | 33 +++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java index d519a8c86266..803ddb6c1ec3 100644 --- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java +++ b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java @@ -28,6 +28,7 @@ import java.nio.channels.WritableByteChannel; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; import org.apache.beam.sdk.io.FileSystem; import org.apache.beam.sdk.io.fs.CreateOptions; @@ -78,6 +79,12 @@ protected List match(List specs) { for (String spec : specs) { try { FileStatus[] fileStatuses = fileSystem.globStatus(new Path(spec)); + if (fileStatuses == null) { + resultsBuilder.add(MatchResult.create(Status.NOT_FOUND, + Collections.emptyList())); + continue; + } + List metadata = new ArrayList<>(); for (FileStatus fileStatus : fileStatuses) { if (fileStatus.isFile()) { diff --git a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java index 88275f49b05c..d06142dce84f 100644 --- a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java +++ b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java @@ -19,6 +19,8 @@ import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -157,6 +159,37 @@ public void testMatch() throws Exception { .build())); } + @Test + public void testMatchForNonExistentFile() throws Exception { + create("testFileAA", "testDataAA".getBytes()); + create("testFileBB", "testDataBB".getBytes()); + + // ensure files exist + assertArrayEquals("testDataAA".getBytes(), read("testFileAA")); + assertArrayEquals("testDataBB".getBytes(), read("testFileBB")); + + List matchResults = fileSystem.match(ImmutableList.of( + testPath("testFileAA").toString(), + testPath("testFileA").toString(), + testPath("testFileBB").toString())); + + assertThat(matchResults, hasSize(3)); + + final List expected = ImmutableList.of( + MatchResult.create(Status.OK, ImmutableList.of(Metadata.builder() + .setResourceId(testPath("testFileAA")) + .setIsReadSeekEfficient(true) + .setSizeBytes("testDataAA".getBytes().length) + .build())), + MatchResult.create(Status.NOT_FOUND, ImmutableList.of()), + MatchResult.create(Status.OK, ImmutableList.of(Metadata.builder() + .setResourceId(testPath("testFileBB")) + .setIsReadSeekEfficient(true) + .setSizeBytes("testDataBB".getBytes().length) + .build()))); + assertThat(matchResults, equalTo(expected)); + } + @Test public void testRename() throws Exception { create("testFileA", "testDataA".getBytes()); From 86912a23795353111934365ed9065a2a71f70074 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 25 May 2017 13:49:48 -0700 Subject: [PATCH 114/159] [BEAM-2365] Use the highest pickle protocol available. --- sdks/python/apache_beam/coders/coders.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index ce914ddec0e0..f40045d142ff 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -365,7 +365,7 @@ def maybe_dill_dumps(o): # We need to use the dill pickler for objects of certain custom classes, # including, for example, ones that contain lambdas. try: - return pickle.dumps(o) + return pickle.dumps(o, pickle.HIGHEST_PROTOCOL) except Exception: # pylint: disable=broad-except return dill.dumps(o) @@ -426,7 +426,10 @@ class PickleCoder(_PickleCoderBase): """Coder using Python's pickle functionality.""" def _create_impl(self): - return coder_impl.CallbackCoderImpl(pickle.dumps, pickle.loads) + dumps = pickle.dumps + HIGHEST_PROTOCOL = pickle.HIGHEST_PROTOCOL + return coder_impl.CallbackCoderImpl( + lambda x: dumps(x, HIGHEST_PROTOCOL), pickle.loads) class DillCoder(_PickleCoderBase): @@ -515,7 +518,7 @@ class Base64PickleCoder(Coder): # than via a special Coder. def encode(self, value): - return base64.b64encode(pickle.dumps(value)) + return base64.b64encode(pickle.dumps(value, pickle.HIGHEST_PROTOCOL)) def decode(self, encoded): return pickle.loads(base64.b64decode(encoded)) From f6ed520fbcb1f2d4b13800b597ecd26eff9c5a08 Mon Sep 17 00:00:00 2001 From: Charles Chen Date: Fri, 26 May 2017 14:24:55 -0700 Subject: [PATCH 115/159] Fix Python Dataflow execution errors due to #3223 --- sdks/python/apache_beam/io/gcp/pubsub.py | 74 +++++++++++++++++++++++- 1 file changed, 73 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/gcp/pubsub.py b/sdks/python/apache_beam/io/gcp/pubsub.py index 7b838d23fea6..1ba8ac051272 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/gcp/pubsub.py @@ -18,10 +18,13 @@ Cloud Pub/Sub sources and sinks are currently supported only in streaming pipelines, during remote execution. + +This API is currently under development and is subject to change. """ from __future__ import absolute_import +from apache_beam import coders from apache_beam.io.iobase import Read from apache_beam.io.iobase import Write from apache_beam.runners.dataflow.native_io import iobase as dataflow_io @@ -30,7 +33,8 @@ from apache_beam.transforms.display import DisplayDataItem -__all__ = ['ReadStringsFromPubSub', 'WriteStringsToPubSub'] +__all__ = ['ReadStringsFromPubSub', 'WriteStringsToPubSub', + 'PubSubSource', 'PubSubSink'] class ReadStringsFromPubSub(PTransform): @@ -150,3 +154,71 @@ def _decodeUtf8String(encoded_value): def _encodeUtf8String(value): """Encodes a string in utf-8 format to bytes""" return value.encode('utf-8') + + +class PubSubSource(dataflow_io.NativeSource): + """Deprecated: do not use. + + Source for reading from a given Cloud Pub/Sub topic. + + Attributes: + topic: Cloud Pub/Sub topic in the form "/topics//". + subscription: Optional existing Cloud Pub/Sub subscription to use in the + form "projects//subscriptions/". + id_label: The attribute on incoming Pub/Sub messages to use as a unique + record identifier. When specified, the value of this attribute (which can + be any string that uniquely identifies the record) will be used for + deduplication of messages. If not provided, Dataflow cannot guarantee + that no duplicate data will be delivered on the Pub/Sub stream. In this + case, deduplication of the stream will be strictly best effort. + coder: The Coder to use for decoding incoming Pub/Sub messages. + """ + + def __init__(self, topic, subscription=None, id_label=None, + coder=coders.StrUtf8Coder()): + self.topic = topic + self.subscription = subscription + self.id_label = id_label + self.coder = coder + + @property + def format(self): + """Source format name required for remote execution.""" + return 'pubsub' + + def display_data(self): + return {'id_label': + DisplayDataItem(self.id_label, + label='ID Label Attribute').drop_if_none(), + 'topic': + DisplayDataItem(self.topic, + label='Pubsub Topic'), + 'subscription': + DisplayDataItem(self.subscription, + label='Pubsub Subscription').drop_if_none()} + + def reader(self): + raise NotImplementedError( + 'PubSubSource is not supported in local execution.') + + +class PubSubSink(dataflow_io.NativeSink): + """Deprecated: do not use. + + Sink for writing to a given Cloud Pub/Sub topic.""" + + def __init__(self, topic, coder=coders.StrUtf8Coder()): + self.topic = topic + self.coder = coder + + @property + def format(self): + """Sink format name required for remote execution.""" + return 'pubsub' + + def display_data(self): + return {'topic': DisplayDataItem(self.topic, label='Pubsub Topic')} + + def writer(self): + raise NotImplementedError( + 'PubSubSink is not supported in local execution.') From 320756df97fbdab2fda732cfe66831025d502af8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Sat, 27 May 2017 21:21:37 +0200 Subject: [PATCH 116/159] Upgrade Avro dependency to version 1.8.2 --- pom.xml | 2 +- .../test/java/org/apache/beam/sdk/coders/AvroCoderTest.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 398cda09d9d3..3e302e750864 100644 --- a/pom.xml +++ b/pom.xml @@ -105,7 +105,7 @@ 1.9 2.24.0 1.0.0-rc2 - 1.8.1 + 1.8.2 v2-rev295-1.22.0 0.9.6.2 v1-rev6-1.22.0 diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java index 8942a9e356ae..60b32327f27a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java @@ -44,7 +44,7 @@ import java.util.SortedSet; import java.util.TreeMap; import java.util.TreeSet; -import org.apache.avro.AvroTypeException; +import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; import org.apache.avro.generic.GenericData; @@ -502,7 +502,7 @@ public void testAvroProhibitsShadowing() { try { ReflectData.get().getSchema(SubclassHidingParent.class); fail("Expected AvroTypeException"); - } catch (AvroTypeException e) { + } catch (AvroRuntimeException e) { assertThat(e.getMessage(), containsString("mapField")); assertThat(e.getMessage(), containsString("two fields named")); } From 9afe395bbddd2382c5222dd3145a0be3cdf7077a Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 30 May 2017 10:56:56 +0200 Subject: [PATCH 117/159] [BEAM-2380] Forward additional outputs to DoFnRunner in Flink Batch --- .../flink/translation/functions/FlinkDoFnFunction.java | 8 +++++--- .../translation/functions/FlinkStatefulDoFnFunction.java | 8 +++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index 9205bce1a83a..42a8833fea7b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -17,7 +17,8 @@ */ package org.apache.beam.runners.flink.translation.functions; -import java.util.Collections; +import com.google.common.collect.Lists; +import java.util.List; import java.util.Map; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; @@ -97,13 +98,14 @@ public void mapPartition( new FlinkDoFnFunction.MultiDoFnOutputManager((Collector) out, outputMap); } + List> additionalOutputTags = Lists.newArrayList(outputMap.keySet()); + DoFnRunner doFnRunner = DoFnRunners.simpleRunner( serializedOptions.getPipelineOptions(), doFn, new FlinkSideInputReader(sideInputs, runtimeContext), outputManager, mainOutputTag, - // see SimpleDoFnRunner, just use it to limit number of additional outputs - Collections.>emptyList(), + additionalOutputTags, new FlinkNoOpStepContext(), windowingStrategy); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java index 6517bf2c1a9f..b07576893d01 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java @@ -19,8 +19,9 @@ import static org.apache.flink.util.Preconditions.checkArgument; -import java.util.Collections; +import com.google.common.collect.Lists; import java.util.Iterator; +import java.util.List; import java.util.Map; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; @@ -114,13 +115,14 @@ public void reduce( timerInternals.advanceProcessingTime(Instant.now()); timerInternals.advanceSynchronizedProcessingTime(Instant.now()); + List> additionalOutputTags = Lists.newArrayList(outputMap.keySet()); + DoFnRunner, OutputT> doFnRunner = DoFnRunners.simpleRunner( serializedOptions.getPipelineOptions(), dofn, new FlinkSideInputReader(sideInputs, runtimeContext), outputManager, mainOutputTag, - // see SimpleDoFnRunner, just use it to limit number of additional outputs - Collections.>emptyList(), + additionalOutputTags, new FlinkNoOpStepContext() { @Override public StateInternals stateInternals() { From 838035a4069b152143859e9b34570b15254d69b3 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 30 May 2017 15:19:27 +0200 Subject: [PATCH 118/159] Fix flushing of pushed-back data in Flink Runner on +Inf watermark Before, we only flushed all pushed-back data when receiving a +Inf watermark on the side input. It can happen that we receive that watermark before getting any data on the main input. This changes DoFnOperator to also flush when receiving a main-input watermark and we happen to have already received the +Inf watermark on the side input. Some tests where Flaky because of this. One example was ViewTest.testWindowedSideInputFixedToFixedWithDefault(). --- .../wrappers/streaming/DoFnOperator.java | 77 +++++++++++++++---- 1 file changed, 60 insertions(+), 17 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index d2ab7e1cabbd..e47304690b1c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -19,6 +19,7 @@ import static org.apache.flink.util.Preconditions.checkArgument; +import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.collect.Iterables; import java.io.DataInputStream; @@ -129,6 +130,8 @@ public class DoFnOperator protected transient long currentInputWatermark; + protected transient long currentSideInputWatermark; + protected transient long currentOutputWatermark; private transient StateTag>> pushedBackTag; @@ -197,6 +200,7 @@ public void open() throws Exception { super.open(); setCurrentInputWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); + setCurrentSideInputWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); setCurrentOutputWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); sideInputReader = NullSideInputReader.of(sideInputs); @@ -308,6 +312,21 @@ public void open() throws Exception { @Override public void close() throws Exception { super.close(); + + // sanity check: these should have been flushed out by +Inf watermarks + if (pushbackStateInternals != null) { + BagState> pushedBack = + pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag); + + Iterable> pushedBackContents = pushedBack.read(); + if (pushedBackContents != null) { + if (!Iterables.isEmpty(pushedBackContents)) { + String pushedBackString = Joiner.on(",").join(pushedBackContents); + throw new RuntimeException( + "Leftover pushed-back data: " + pushedBackString + ". This indicates a bug."); + } + } + } doFnInvoker.invokeTeardown(); } @@ -457,36 +476,56 @@ public void processWatermark1(Watermark mark) throws Exception { } pushbackDoFnRunner.finishBundle(); } + + // We do the check here because we are guaranteed to at least get the +Inf watermark on the + // main input when the job finishes. + if (currentSideInputWatermark >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { + // this means we will never see any more side input + // we also do the check here because we might have received the side-input MAX watermark + // before receiving any main-input data + emitAllPushedBackData(); + } } @Override public void processWatermark2(Watermark mark) throws Exception { - if (mark.getTimestamp() == BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { + setCurrentSideInputWatermark(mark.getTimestamp()); + if (mark.getTimestamp() >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { // this means we will never see any more side input - pushbackDoFnRunner.startBundle(); + emitAllPushedBackData(); - BagState> pushedBack = - pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag); + // maybe output a new watermark + processWatermark1(new Watermark(currentInputWatermark)); + } + } - Iterable> pushedBackContents = pushedBack.read(); - if (pushedBackContents != null) { - for (WindowedValue elem : pushedBackContents) { + /** + * Emits all pushed-back data. This should be used once we know that there will not be + * any future side input, i.e. that there is no point in waiting. + */ + private void emitAllPushedBackData() throws Exception { + pushbackDoFnRunner.startBundle(); - // we need to set the correct key in case the operator is - // a (keyed) window operator - setKeyContextElement1(new StreamRecord<>(elem)); + BagState> pushedBack = + pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag); - doFnRunner.processElement(elem); - } + Iterable> pushedBackContents = pushedBack.read(); + if (pushedBackContents != null) { + for (WindowedValue elem : pushedBackContents) { + + // we need to set the correct key in case the operator is + // a (keyed) window operator + setKeyContextElement1(new StreamRecord<>(elem)); + + doFnRunner.processElement(elem); } + } - setPushedBackWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()); + pushedBack.clear(); - pushbackDoFnRunner.finishBundle(); + setPushedBackWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()); - // maybe output a new watermark - processWatermark1(new Watermark(currentInputWatermark)); - } + pushbackDoFnRunner.finishBundle(); } @Override @@ -610,6 +649,10 @@ private void setCurrentInputWatermark(long currentInputWatermark) { this.currentInputWatermark = currentInputWatermark; } + private void setCurrentSideInputWatermark(long currentInputWatermark) { + this.currentSideInputWatermark = currentInputWatermark; + } + private void setCurrentOutputWatermark(long currentOutputWatermark) { this.currentOutputWatermark = currentOutputWatermark; } From dd79d5aac3b44a6cb9569d6fb4f30e10335ab66d Mon Sep 17 00:00:00 2001 From: James Xu Date: Tue, 30 May 2017 11:27:32 +0800 Subject: [PATCH 119/159] fix javadoc of View --- .../src/main/java/org/apache/beam/sdk/transforms/View.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java index bcbdb24d5290..073c750901da 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java @@ -116,16 +116,17 @@ * {@code * PCollection pages = ... // pages fit into memory * PCollection urlVisits = ... // very large collection - * final PCollectionView> = urlToPage + * final PCollectionView> urlToPageView = pages * .apply(WithKeys.of( ... )) // extract the URL from the page * .apply(View.asMap()); * * PCollection PageVisits = urlVisits - * .apply(ParDo.withSideInputs(urlToPage) + * .apply(ParDo.withSideInputs(urlToPageView) * .of(new DoFn() { * {@literal @}Override * void processElement(ProcessContext context) { * UrlVisit urlVisit = context.element(); + * Map urlToPage = context.sideInput(urlToPageView); * Page page = urlToPage.get(urlVisit.getUrl()); * c.output(new PageVisit(page, urlVisit.getVisitData())); * } From 84a831e3f3ea7f476a28778468cb084f5eecac68 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Tue, 30 May 2017 11:54:55 -0700 Subject: [PATCH 120/159] [BEAM-2379] Avoid reading projectId from environment variable in tests. SpannerOptions.Builder requires projectId to be not-null. --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index c5325bb5a85f..5058d13f77bb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -149,14 +149,6 @@ abstract static class Builder { abstract Write build(); } - SpannerOptions getSpannerOptions() { - SpannerOptions.Builder builder = SpannerOptions.newBuilder(); - if (getServiceFactory() != null) { - builder.setServiceFactory(getServiceFactory()); - } - return builder.build(); - } - /** * Returns a new {@link SpannerIO.Write} that will write to the specified Cloud Spanner project. * @@ -259,10 +251,10 @@ static class SpannerWriteFn extends DoFn { @Setup public void setup() throws Exception { - spanner = spec.getSpannerOptions().getService(); - dbClient = - spanner.getDatabaseClient( - DatabaseId.of(projectId(), spec.getInstanceId(), spec.getDatabaseId())); + SpannerOptions spannerOptions = getSpannerOptions(); + spanner = spannerOptions.getService(); + dbClient = spanner.getDatabaseClient( + DatabaseId.of(projectId(), spec.getInstanceId(), spec.getDatabaseId())); mutations = new ArrayList<>(); batchSizeBytes = 0; } @@ -299,6 +291,17 @@ public void teardown() throws Exception { spanner = null; } + private SpannerOptions getSpannerOptions() { + SpannerOptions.Builder spannerOptionsBuider = SpannerOptions.newBuilder(); + if (spec.getServiceFactory() != null) { + spannerOptionsBuider.setServiceFactory(spec.getServiceFactory()); + } + if (spec.getProjectId() != null) { + spannerOptionsBuider.setProjectId(spec.getProjectId()); + } + return spannerOptionsBuider.build(); + } + /** * Writes a batch of mutations to Cloud Spanner. * From 60779e2ecd76f1cb4766050e4560765c1bc3c19b Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Tue, 30 May 2017 13:15:31 -0700 Subject: [PATCH 121/159] [BEAM-1347] Remove the usage of a thread local on a potentially hot path --- .../harness/logging/BeamFnLoggingClient.java | 36 ++++++++++--------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java index c8d11ed8ff91..d56ee6d6ae95 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java @@ -38,7 +38,6 @@ import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; -import java.util.function.Consumer; import java.util.function.Function; import java.util.logging.Formatter; import java.util.logging.Handler; @@ -179,11 +178,14 @@ private class LogRecordHandler extends Handler implements Runnable { private final BlockingDeque bufferedLogEntries = new LinkedBlockingDeque<>(MAX_BUFFERED_LOG_ENTRY_COUNT); private final Future bufferedLogWriter; - private final ThreadLocal> logEntryHandler; + /** + * Safe object publishing is not required since we only care if the thread that set + * this field is equal to the thread also attempting to add a log entry. + */ + private Thread logEntryHandlerThread; private LogRecordHandler(ExecutorService executorService) { bufferedLogWriter = executorService.submit(this); - logEntryHandler = new ThreadLocal<>(); } @Override @@ -204,19 +206,18 @@ public void publish(LogRecord record) { builder.setTrace(getStackTraceAsString(record.getThrown())); } // The thread that sends log records should never perform a blocking publish and - // only insert log records best effort. We detect which thread is logging - // by using the thread local, defaulting to the blocking publish. - MoreObjects.firstNonNull( - logEntryHandler.get(), this::blockingPublish).accept(builder.build()); - } - - /** Blocks caller till enough space exists to publish this log entry. */ - private void blockingPublish(BeamFnApi.LogEntry logEntry) { - try { - bufferedLogEntries.put(logEntry); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); + // only insert log records best effort. + if (Thread.currentThread() != logEntryHandlerThread) { + // Blocks caller till enough space exists to publish this log entry. + try { + bufferedLogEntries.put(builder.build()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } else { + // Never blocks caller, will drop log message if buffer is full. + bufferedLogEntries.offer(builder.build()); } } @@ -225,7 +226,8 @@ public void run() { // Logging which occurs in this thread will attempt to publish log entries into the // above handler which should never block if the queue is full otherwise // this thread will get stuck. - logEntryHandler.set(bufferedLogEntries::offer); + logEntryHandlerThread = Thread.currentThread(); + List additionalLogEntries = new ArrayList<>(MAX_BUFFERED_LOG_ENTRY_COUNT); try { From d29e353ea53349e3c94285fdf5b29318252087d1 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Wed, 10 May 2017 19:49:04 +0800 Subject: [PATCH 122/159] [BEAM-2248] KafkaIO support to use start read time to set start offset --- sdks/java/io/kafka/pom.xml | 2 +- .../beam/sdk/io/kafka/ConsumerSpEL.java | 56 ++++++++ .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 44 ++++++- .../apache/beam/sdk/io/kafka/KafkaIOTest.java | 122 ++++++++++++++++-- 4 files changed, 205 insertions(+), 19 deletions(-) diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml index f6f038513f86..29350ccfd431 100644 --- a/sdks/java/io/kafka/pom.xml +++ b/sdks/java/io/kafka/pom.xml @@ -30,7 +30,7 @@ Library to read Kafka topics. - 0.9.0.1 + 0.10.1.0 diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConsumerSpEL.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConsumerSpEL.java index 8fe17c1e8770..8cdad228f210 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConsumerSpEL.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConsumerSpEL.java @@ -17,12 +17,18 @@ */ package org.apache.beam.sdk.io.kafka; +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.Collection; +import java.util.Map; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.expression.Expression; @@ -51,13 +57,28 @@ class ConsumerSpEL { private Method timestampMethod; private boolean hasRecordTimestamp = false; + private Method offsetGetterMethod; + private Method offsetsForTimesMethod; + private boolean hasOffsetsForTimes = false; + public ConsumerSpEL() { try { + // It is supported by Kafka Client 0.10.0.0 onwards. timestampMethod = ConsumerRecord.class.getMethod("timestamp", (Class[]) null); hasRecordTimestamp = timestampMethod.getReturnType().equals(Long.TYPE); } catch (NoSuchMethodException | SecurityException e) { LOG.debug("Timestamp for Kafka message is not available."); } + + try { + // It is supported by Kafka Client 0.10.1.0 onwards. + offsetGetterMethod = Class.forName("org.apache.kafka.clients.consumer.OffsetAndTimestamp") + .getMethod("offset", (Class[]) null); + offsetsForTimesMethod = Consumer.class.getMethod("offsetsForTimes", Map.class); + hasOffsetsForTimes = offsetsForTimesMethod.getReturnType().equals(Map.class); + } catch (NoSuchMethodException | SecurityException | ClassNotFoundException e) { + LOG.debug("OffsetsForTimes is not available."); + } } public void evaluateSeek2End(Consumer consumer, TopicPartition topicPartitions) { @@ -88,4 +109,39 @@ public long getRecordTimestamp(ConsumerRecord rawRecord) { } return timestamp; } + + public boolean hasOffsetsForTimes() { + return hasOffsetsForTimes; + } + + /** + * Look up the offset for the given partition by timestamp. + * Throws RuntimeException if there are no messages later than timestamp or if this partition + * does not support timestamp based offset. + */ + @SuppressWarnings("unchecked") + public long offsetForTime(Consumer consumer, TopicPartition topicPartition, Instant time) { + + checkArgument(hasOffsetsForTimes, + "This Kafka Client must support Consumer.OffsetsForTimes()."); + + Map timestampsToSearch = + ImmutableMap.of(topicPartition, time.getMillis()); + try { + Map offsetsByTimes = (Map) offsetsForTimesMethod.invoke(consumer, timestampsToSearch); + Object offsetAndTimestamp = Iterables.getOnlyElement(offsetsByTimes.values()); + + if (offsetAndTimestamp == null) { + throw new RuntimeException("There are no messages has a timestamp that is greater than or " + + "equals to the target time or the message format version in this partition is " + + "before 0.10.0, topicPartition is: " + topicPartition); + } else { + return (long) offsetGetterMethod.invoke(offsetAndTimestamp); + } + } catch (IllegalAccessException | InvocationTargetException e) { + throw new RuntimeException(e); + } + + } + } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index a1130fc15e3f..4d2a3584f231 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -101,6 +101,7 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.AppInfoParser; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -306,6 +307,8 @@ public abstract static class Read abstract long getMaxNumRecords(); @Nullable abstract Duration getMaxReadTime(); + @Nullable abstract Instant getStartReadTime(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -324,6 +327,7 @@ abstract Builder setConsumerFactoryFn( abstract Builder setWatermarkFn(SerializableFunction, Instant> fn); abstract Builder setMaxNumRecords(long maxNumRecords); abstract Builder setMaxReadTime(Duration maxReadTime); + abstract Builder setStartReadTime(Instant startReadTime); abstract Read build(); } @@ -447,6 +451,24 @@ public Read withMaxNumRecords(long maxNumRecords) { return toBuilder().setMaxNumRecords(maxNumRecords).setMaxReadTime(null).build(); } + /** + * Use timestamp to set up start offset. + * It is only supported by Kafka Client 0.10.1.0 onwards and the message format version + * after 0.10.0. + * + *

Note that this take priority over start offset configuration + * {@code ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} and any auto committed offsets. + * + *

This results in hard failures in either of the following two cases : + * 1. If one of more partitions do not contain any messages with timestamp larger than or + * equal to desired timestamp. + * 2. If the message format version in a partition is before 0.10.0, i.e. the messages do + * not have timestamps. + */ + public Read withStartReadTime(Instant startReadTime) { + return toBuilder().setStartReadTime(startReadTime).build(); + } + /** * Similar to * {@link org.apache.beam.sdk.io.Read.Unbounded#withMaxReadTime(Duration)}. @@ -508,6 +530,13 @@ public void validate(PipelineOptions options) { "Kafka topics or topic_partitions are required"); checkNotNull(getKeyDeserializer(), "Key deserializer must be set"); checkNotNull(getValueDeserializer(), "Value deserializer must be set"); + if (getStartReadTime() != null) { + checkArgument(new ConsumerSpEL().hasOffsetsForTimes(), + "Consumer.offsetsForTimes is only supported by Kafka Client 0.10.1.0 onwards, " + + "current version of Kafka Client is " + AppInfoParser.getVersion() + + ". If you are building with maven, set \"kafka.clients.version\" " + + "maven property to 0.10.1.0 or newer."); + } } @Override @@ -1041,10 +1070,17 @@ public boolean start() throws IOException { consumer.seek(p.topicPartition, p.nextOffset); } else { // nextOffset is unininitialized here, meaning start reading from latest record as of now - // ('latest' is the default, and is configurable). Remember the current position without - // waiting until the first record read. This ensures checkpoint is accurate even if the - // reader is closed before reading any records. - p.nextOffset = consumer.position(p.topicPartition); + // ('latest' is the default, and is configurable) or 'look up offset by startReadTime. + // Remember the current position without waiting until the first record is read. This + // ensures checkpoint is accurate even if the reader is closed before reading any records. + Instant startReadTime = spec.getStartReadTime(); + if (startReadTime != null) { + p.nextOffset = + consumerSpEL.offsetForTime(consumer, p.topicPartition, spec.getStartReadTime()); + consumer.seek(p.topicPartition, p.nextOffset); + } else { + p.nextOffset = consumer.position(p.topicPartition); + } } LOG.info("{}: reading from {} starting at offset {}", name, p.topicPartition, p.nextOffset); diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index 691f7f4fdc4f..b69bc83561fc 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -24,11 +24,14 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import static org.junit.Assume.assumeTrue; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; @@ -150,7 +153,7 @@ private static MockConsumer mkMockConsumer( } int numPartitions = partitions.size(); - long[] offsets = new long[numPartitions]; + final long[] offsets = new long[numPartitions]; for (int i = 0; i < numElements; i++) { int pIdx = i % numPartitions; @@ -184,6 +187,36 @@ public void assign(final List assigned) { updateEndOffsets(ImmutableMap.of(tp, (long) records.get(tp).size())); } } + // Override offsetsForTimes() in order to look up the offsets by timestamp. + // Remove keyword '@Override' here, Kafka client 0.10.1.0 previous versions does not have + // this method. + // Should return Map, but 0.10.1.0 previous versions + // does not have the OffsetAndTimestamp class. So return a raw type and use reflection + // here. + @SuppressWarnings("unchecked") + public Map offsetsForTimes(Map timestampsToSearch) { + HashMap result = new HashMap<>(); + try { + Class cls = Class.forName("org.apache.kafka.clients.consumer.OffsetAndTimestamp"); + // OffsetAndTimestamp(long offset, long timestamp) + Constructor constructor = cls.getDeclaredConstructor(long.class, long.class); + + // In test scope, timestamp == offset. + for (Map.Entry entry : timestampsToSearch.entrySet()) { + long maxOffset = offsets[partitions.indexOf(entry.getKey())]; + Long offset = entry.getValue(); + if (offset >= maxOffset) { + offset = null; + } + result.put( + entry.getKey(), constructor.newInstance(entry.getValue(), offset)); + } + return result; + } catch (ClassNotFoundException | IllegalAccessException + | InstantiationException | NoSuchMethodException | InvocationTargetException e) { + throw new RuntimeException(e); + } + } }; for (String topic : topics) { @@ -239,12 +272,19 @@ public Consumer apply(Map config) { } } + private static KafkaIO.Read mkKafkaReadTransform( + int numElements, + @Nullable SerializableFunction, Instant> timestampFn) { + return mkKafkaReadTransform(numElements, numElements, timestampFn); + } + /** * Creates a consumer with two topics, with 10 partitions each. * numElements are (round-robin) assigned all the 20 partitions. */ private static KafkaIO.Read mkKafkaReadTransform( int numElements, + int maxNumRecords, @Nullable SerializableFunction, Instant> timestampFn) { List topics = ImmutableList.of("topic_a", "topic_b"); @@ -256,7 +296,7 @@ private static KafkaIO.Read mkKafkaReadTransform( topics, 10, numElements, OffsetResetStrategy.EARLIEST)) // 20 partitions .withKeyDeserializer(IntegerDeserializer.class) .withValueDeserializer(LongDeserializer.class) - .withMaxNumRecords(numElements); + .withMaxNumRecords(maxNumRecords); if (timestampFn != null) { return reader.withTimestampFn(timestampFn); @@ -283,22 +323,31 @@ public Void apply(Iterable values) { public static void addCountingAsserts(PCollection input, long numElements) { // Count == numElements - PAssert - .thatSingleton(input.apply("Count", Count.globally())) - .isEqualTo(numElements); // Unique count == numElements - PAssert - .thatSingleton(input.apply(Distinct.create()) - .apply("UniqueCount", Count.globally())) - .isEqualTo(numElements); // Min == 0 - PAssert - .thatSingleton(input.apply("Min", Min.globally())) - .isEqualTo(0L); // Max == numElements-1 + addCountingAsserts(input, numElements, numElements, 0L, numElements - 1); + } + + public static void addCountingAsserts( + PCollection input, long count, long uniqueCount, long min, long max) { + + PAssert + .thatSingleton(input.apply("Count", Count.globally())) + .isEqualTo(count); + + PAssert + .thatSingleton(input.apply(Distinct.create()) + .apply("UniqueCount", Count.globally())) + .isEqualTo(uniqueCount); + PAssert - .thatSingleton(input.apply("Max", Max.globally())) - .isEqualTo(numElements - 1); + .thatSingleton(input.apply("Min", Min.globally())) + .isEqualTo(min); + + PAssert + .thatSingleton(input.apply("Max", Max.globally())) + .isEqualTo(max); } @Test @@ -748,6 +797,51 @@ public void testSinkWithSendErrors() throws Throwable { } } + @Test + public void testUnboundedSourceStartReadTime() { + + assumeTrue(new ConsumerSpEL().hasOffsetsForTimes()); + + int numElements = 1000; + // In this MockConsumer, we let the elements of the time and offset equal and there are 20 + // partitions. So set this startTime can read half elements. + int startTime = numElements / 20 / 2; + int maxNumRecords = numElements / 2; + + PCollection input = p + .apply(mkKafkaReadTransform(numElements, maxNumRecords, new ValueAsTimestampFn()) + .withStartReadTime(new Instant(startTime)) + .withoutMetadata()) + .apply(Values.create()); + + addCountingAsserts(input, maxNumRecords, maxNumRecords, maxNumRecords, numElements - 1); + p.run(); + + } + + @Rule public ExpectedException noMessagesException = ExpectedException.none(); + + @Test + public void testUnboundedSourceStartReadTimeException() { + + assumeTrue(new ConsumerSpEL().hasOffsetsForTimes()); + + noMessagesException.expect(RuntimeException.class); + + int numElements = 1000; + // In this MockConsumer, we let the elements of the time and offset equal and there are 20 + // partitions. So set this startTime can not read any element. + int startTime = numElements / 20; + + p.apply(mkKafkaReadTransform(numElements, numElements, new ValueAsTimestampFn()) + .withStartReadTime(new Instant(startTime)) + .withoutMetadata()) + .apply(Values.create()); + + p.run(); + + } + @Test public void testSourceDisplayData() { KafkaIO.Read read = mkKafkaReadTransform(10, null); From 2ddaaeac1462e88bb447802f7574d4a4cee12ed0 Mon Sep 17 00:00:00 2001 From: Sourabh Bajaj Date: Tue, 30 May 2017 14:19:25 -0700 Subject: [PATCH 123/159] Clean up pyc files before running tests --- sdks/python/tox.ini | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 917e907b5ce2..2166f6a4ee80 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -31,6 +31,8 @@ deps = grpcio-tools==1.3.5 commands = python --version + # Clean up all previous python generated files. + - find apache_beam -type f -name '*.pyc' -delete pip install -e .[test] python apache_beam/examples/complete/autocomplete_test.py python setup.py test @@ -50,6 +52,8 @@ deps = whitelist_externals=find commands = python --version + # Clean up all previous python generated files. + - find apache_beam -type f -name '*.pyc' -delete # Clean up all previous cython generated files. - find apache_beam -type f -name '*.c' -delete - find apache_beam -type f -name '*.so' -delete @@ -72,6 +76,8 @@ deps = commands = pip install -e .[test,gcp] python --version + # Clean up all previous python generated files. + - find apache_beam -type f -name '*.pyc' -delete python apache_beam/examples/complete/autocomplete_test.py python setup.py test passenv = TRAVIS* From 2f9a38e5fa7afa9cd4751980e6cbb989e260aaa2 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Fri, 19 May 2017 11:24:15 -0700 Subject: [PATCH 124/159] [BEAM-1544] Java cross-JDK version tests on Jenkins --- .../jenkins/common_job_properties.groovy | 4 +- ...eam_PostCommit_Java_JDKVersionsTest.groovy | 60 +++++++++++++++++++ 2 files changed, 63 insertions(+), 1 deletion(-) create mode 100644 .test-infra/jenkins/job_beam_PostCommit_Java_JDKVersionsTest.groovy diff --git a/.test-infra/jenkins/common_job_properties.groovy b/.test-infra/jenkins/common_job_properties.groovy index f47ab282a698..5ba12fe70e14 100644 --- a/.test-infra/jenkins/common_job_properties.groovy +++ b/.test-infra/jenkins/common_job_properties.groovy @@ -188,7 +188,9 @@ class common_job_properties { // Disable archiving the built artifacts by default, as this is slow and flaky. // We can usually recreate them easily, and we can also opt-in individual jobs // to artifact archiving. - context.archivingDisabled(true) + if (context.metaClass.respondsTo(context, 'archivingDisabled', boolean)) { + context.archivingDisabled(true) + } } // Sets common config for PreCommit jobs. diff --git a/.test-infra/jenkins/job_beam_PostCommit_Java_JDKVersionsTest.groovy b/.test-infra/jenkins/job_beam_PostCommit_Java_JDKVersionsTest.groovy new file mode 100644 index 000000000000..f23e741a00ef --- /dev/null +++ b/.test-infra/jenkins/job_beam_PostCommit_Java_JDKVersionsTest.groovy @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import common_job_properties + +// This job runs the Java postcommit tests cross multiple JDK versions. +matrixJob('beam_PostCommit_Java_JDK_Versions_Test') { + description('Runs postcommit tests on the Java SDK in multiple Jdk versions.') + + // Set common parameters. + common_job_properties.setTopLevelMainJobProperties(delegate) + + // Set JDK versions. + axes { + label('label', 'beam') + jdk('JDK 1.7 (latest)', + 'OpenJDK 7 (on Ubuntu only)', + 'OpenJDK 8 (on Ubuntu only)') + } + + // Sets that this is a PostCommit job. + common_job_properties.setPostCommit( + delegate, + '0 */6 * * *', + false) + + // Allows triggering this build against pull requests. + common_job_properties.enablePhraseTriggeringFromPullRequest( + delegate, + 'Java JDK Version Test', + 'Run Java JDK Version Test') + + // Maven build for this job. + steps { + maven { + // Set maven parameters. + common_job_properties.setMavenConfig(delegate) + + // Maven build project. + // Skip beam-sdks-python since this test is only apply to Java. + // TODO[BEAM-2322,BEAM-2323,BEAM-2324]: Re-enable beam-runners-apex once the build is passed. + goals('-B -e -P dataflow-runner clean install -pl \'!org.apache.beam:beam-sdks-python,!org.apache.beam:beam-runners-apex\' -DskipITs=false -DintegrationTestPipelineOptions=\'[ "--project=apache-beam-testing", "--tempRoot=gs://temp-storage-for-end-to-end-tests", "--runner=TestDataflowRunner" ]\'') + } + } +} From 636eaff03646113daf868949734199f5697bdf0d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 2 May 2017 01:33:27 +0200 Subject: [PATCH 125/159] Refactor HadoopInputFormatIO to use SerializableConfiguration from hadoop-common --- .../inputformat/HadoopInputFormatIO.java | 53 ++---------- .../inputformat/HadoopInputFormatIOTest.java | 80 +++++++++---------- 2 files changed, 47 insertions(+), 86 deletions(-) diff --git a/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java b/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java index 336740c65e76..efd47fd85c4c 100644 --- a/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java +++ b/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java @@ -23,11 +23,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.util.concurrent.AtomicDouble; -import java.io.Externalizable; import java.io.IOException; -import java.io.ObjectInput; import java.io.ObjectInputStream; -import java.io.ObjectOutput; import java.io.ObjectOutputStream; import java.io.Serializable; import java.math.BigDecimal; @@ -46,6 +43,7 @@ import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.hadoop.SerializableConfiguration; import org.apache.beam.sdk.io.hadoop.WritableCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.PTransform; @@ -432,7 +430,7 @@ public void validate() { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - Configuration hadoopConfig = getConfiguration().getHadoopConfiguration(); + Configuration hadoopConfig = getConfiguration().get(); if (hadoopConfig != null) { builder.addIfNotNull(DisplayData.item("mapreduce.job.inputformat.class", hadoopConfig.get("mapreduce.job.inputformat.class")) @@ -493,7 +491,7 @@ void computeSplitsIfNecessary() throws IOException, InterruptedException { } createInputFormatInstance(); List splits = - inputFormatObj.getSplits(Job.getInstance(conf.getHadoopConfiguration())); + inputFormatObj.getSplits(Job.getInstance(conf.get())); if (splits == null) { throw new IOException("Error in computing splits, getSplits() returns null."); } @@ -520,12 +518,12 @@ protected void createInputFormatInstance() throws IOException { if (inputFormatObj == null) { try { taskAttemptContext = - new TaskAttemptContextImpl(conf.getHadoopConfiguration(), new TaskAttemptID()); + new TaskAttemptContextImpl(conf.get(), new TaskAttemptID()); inputFormatObj = (InputFormat) conf - .getHadoopConfiguration() + .get() .getClassByName( - conf.getHadoopConfiguration().get("mapreduce.job.inputformat.class")) + conf.get().get("mapreduce.job.inputformat.class")) .newInstance(); /* * If InputFormat explicitly implements interface {@link Configurable}, then setConf() @@ -535,7 +533,7 @@ protected void createInputFormatInstance() throws IOException { * org.apache.hadoop.hbase.mapreduce.TableInputFormat TableInputFormat}, etc. */ if (Configurable.class.isAssignableFrom(inputFormatObj.getClass())) { - ((Configurable) inputFormatObj).setConf(conf.getHadoopConfiguration()); + ((Configurable) inputFormatObj).setConf(conf.get()); } } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) { throw new IOException("Unable to create InputFormat object: ", e); @@ -802,41 +800,4 @@ private void writeObject(ObjectOutputStream out) throws IOException { new ObjectWritable(inputSplit).write(out); } } - - /** - * A wrapper to allow Hadoop {@link org.apache.hadoop.conf.Configuration} to be serialized using - * Java's standard serialization mechanisms. Note that the org.apache.hadoop.conf.Configuration - * is Writable. - */ - public static class SerializableConfiguration implements Externalizable { - - private Configuration conf; - - public SerializableConfiguration() {} - - public SerializableConfiguration(Configuration conf) { - this.conf = conf; - } - - public Configuration getHadoopConfiguration() { - return conf; - } - - @Override - public void writeExternal(ObjectOutput out) throws IOException { - out.writeUTF(conf.getClass().getCanonicalName()); - ((Writable) conf).write(out); - } - - @Override - public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - String className = in.readUTF(); - try { - conf = (Configuration) Class.forName(className).newInstance(); - conf.readFields(in); - } catch (InstantiationException | IllegalAccessException e) { - throw new IOException("Unable to create configuration: " + e); - } - } - } } diff --git a/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIOTest.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIOTest.java index aeeeb1761d0f..9ec3838ae54f 100644 --- a/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIOTest.java +++ b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIOTest.java @@ -26,11 +26,11 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.io.hadoop.SerializableConfiguration; import org.apache.beam.sdk.io.hadoop.WritableCoder; import org.apache.beam.sdk.io.hadoop.inputformat.EmployeeInputFormat.EmployeeRecordReader; import org.apache.beam.sdk.io.hadoop.inputformat.EmployeeInputFormat.NewObjectsEmployeeInputSplit; import org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO.HadoopInputFormatBoundedSource; -import org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO.SerializableConfiguration; import org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO.SerializableSplit; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.PAssert; @@ -94,11 +94,11 @@ public String apply(Employee input) { @Test public void testReadBuildsCorrectly() { HadoopInputFormatIO.Read read = HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()) + .withConfiguration(serConf.get()) .withKeyTranslation(myKeyTranslate) .withValueTranslation(myValueTranslate); - assertEquals(serConf.getHadoopConfiguration(), - read.getConfiguration().getHadoopConfiguration()); + assertEquals(serConf.get(), + read.getConfiguration().get()); assertEquals(myKeyTranslate, read.getKeyTranslationFunction()); assertEquals(myValueTranslate, read.getValueTranslationFunction()); assertEquals(myValueTranslate.getOutputTypeDescriptor(), read.getValueTypeDescriptor()); @@ -116,10 +116,10 @@ public void testReadBuildsCorrectlyInDifferentOrder() { HadoopInputFormatIO.Read read = HadoopInputFormatIO.read() .withValueTranslation(myValueTranslate) - .withConfiguration(serConf.getHadoopConfiguration()) + .withConfiguration(serConf.get()) .withKeyTranslation(myKeyTranslate); - assertEquals(serConf.getHadoopConfiguration(), - read.getConfiguration().getHadoopConfiguration()); + assertEquals(serConf.get(), + read.getConfiguration().get()); assertEquals(myKeyTranslate, read.getKeyTranslationFunction()); assertEquals(myValueTranslate, read.getValueTranslationFunction()); assertEquals(myKeyTranslate.getOutputTypeDescriptor(), read.getKeyTypeDescriptor()); @@ -142,15 +142,15 @@ public void testReadBuildsCorrectlyIfWithConfigurationIsCalledMoreThanOneTime() Employee.class, Text.class); HadoopInputFormatIO.Read read = HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()) + .withConfiguration(serConf.get()) .withKeyTranslation(myKeyTranslate) - .withConfiguration(diffConf.getHadoopConfiguration()); - assertEquals(diffConf.getHadoopConfiguration(), - read.getConfiguration().getHadoopConfiguration()); + .withConfiguration(diffConf.get()); + assertEquals(diffConf.get(), + read.getConfiguration().get()); assertEquals(myKeyTranslate, read.getKeyTranslationFunction()); assertEquals(null, read.getValueTranslationFunction()); assertEquals(myKeyTranslate.getOutputTypeDescriptor(), read.getKeyTypeDescriptor()); - assertEquals(diffConf.getHadoopConfiguration().getClass("value.class", Object.class), read + assertEquals(diffConf.get().getClass("value.class", Object.class), read .getValueTypeDescriptor().getRawType()); } @@ -173,14 +173,14 @@ public void testReadObjectCreationFailsIfConfigurationIsNull() { @Test public void testReadObjectCreationWithConfiguration() { HadoopInputFormatIO.Read read = HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()); - assertEquals(serConf.getHadoopConfiguration(), - read.getConfiguration().getHadoopConfiguration()); + .withConfiguration(serConf.get()); + assertEquals(serConf.get(), + read.getConfiguration().get()); assertEquals(null, read.getKeyTranslationFunction()); assertEquals(null, read.getValueTranslationFunction()); - assertEquals(serConf.getHadoopConfiguration().getClass("key.class", Object.class), read + assertEquals(serConf.get().getClass("key.class", Object.class), read .getKeyTypeDescriptor().getRawType()); - assertEquals(serConf.getHadoopConfiguration().getClass("value.class", Object.class), read + assertEquals(serConf.get().getClass("value.class", Object.class), read .getValueTypeDescriptor().getRawType()); } @@ -194,7 +194,7 @@ public void testReadObjectCreationWithConfiguration() { public void testReadObjectCreationFailsIfKeyTranslationFunctionIsNull() { thrown.expect(NullPointerException.class); HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()) + .withConfiguration(serConf.get()) .withKeyTranslation(null); } @@ -205,15 +205,15 @@ public void testReadObjectCreationFailsIfKeyTranslationFunctionIsNull() { @Test public void testReadObjectCreationWithConfigurationKeyTranslation() { HadoopInputFormatIO.Read read = HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()) + .withConfiguration(serConf.get()) .withKeyTranslation(myKeyTranslate); - assertEquals(serConf.getHadoopConfiguration(), - read.getConfiguration().getHadoopConfiguration()); + assertEquals(serConf.get(), + read.getConfiguration().get()); assertEquals(myKeyTranslate, read.getKeyTranslationFunction()); assertEquals(null, read.getValueTranslationFunction()); assertEquals(myKeyTranslate.getOutputTypeDescriptor().getRawType(), read.getKeyTypeDescriptor().getRawType()); - assertEquals(serConf.getHadoopConfiguration().getClass("value.class", Object.class), + assertEquals(serConf.get().getClass("value.class", Object.class), read.getValueTypeDescriptor().getRawType()); } @@ -227,7 +227,7 @@ public void testReadObjectCreationWithConfigurationKeyTranslation() { public void testReadObjectCreationFailsIfValueTranslationFunctionIsNull() { thrown.expect(NullPointerException.class); HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()) + .withConfiguration(serConf.get()) .withValueTranslation(null); } @@ -238,13 +238,13 @@ public void testReadObjectCreationFailsIfValueTranslationFunctionIsNull() { @Test public void testReadObjectCreationWithConfigurationValueTranslation() { HadoopInputFormatIO.Read read = HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()) + .withConfiguration(serConf.get()) .withValueTranslation(myValueTranslate); - assertEquals(serConf.getHadoopConfiguration(), - read.getConfiguration().getHadoopConfiguration()); + assertEquals(serConf.get(), + read.getConfiguration().get()); assertEquals(null, read.getKeyTranslationFunction()); assertEquals(myValueTranslate, read.getValueTranslationFunction()); - assertEquals(serConf.getHadoopConfiguration().getClass("key.class", Object.class), + assertEquals(serConf.get().getClass("key.class", Object.class), read.getKeyTypeDescriptor().getRawType()); assertEquals(myValueTranslate.getOutputTypeDescriptor().getRawType(), read.getValueTypeDescriptor().getRawType()); @@ -257,11 +257,11 @@ public void testReadObjectCreationWithConfigurationValueTranslation() { @Test public void testReadObjectCreationWithConfigurationKeyTranslationValueTranslation() { HadoopInputFormatIO.Read read = HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()) + .withConfiguration(serConf.get()) .withKeyTranslation(myKeyTranslate) .withValueTranslation(myValueTranslate); - assertEquals(serConf.getHadoopConfiguration(), - read.getConfiguration().getHadoopConfiguration()); + assertEquals(serConf.get(), + read.getConfiguration().get()); assertEquals(myKeyTranslate, read.getKeyTranslationFunction()); assertEquals(myValueTranslate, read.getValueTranslationFunction()); assertEquals(myKeyTranslate.getOutputTypeDescriptor().getRawType(), @@ -342,13 +342,13 @@ public String apply(LongWritable input) { } }; HadoopInputFormatIO.Read read = HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()) + .withConfiguration(serConf.get()) .withKeyTranslation(myKeyTranslateWithWrongInputType); thrown.expect(IllegalArgumentException.class); thrown.expectMessage(String.format( "Key translation's input type is not same as hadoop InputFormat : %s key " + "class : %s", - serConf.getHadoopConfiguration().getClass("mapreduce.job.inputformat.class", - InputFormat.class), serConf.getHadoopConfiguration() + serConf.get().getClass("mapreduce.job.inputformat.class", + InputFormat.class), serConf.get() .getClass("key.class", Object.class))); read.validateTransform(); } @@ -370,15 +370,15 @@ public String apply(LongWritable input) { }; HadoopInputFormatIO.Read read = HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()) + .withConfiguration(serConf.get()) .withValueTranslation(myValueTranslateWithWrongInputType); String expectedMessage = String.format( "Value translation's input type is not same as hadoop InputFormat : " + "%s value class : %s", - serConf.getHadoopConfiguration().getClass("mapreduce.job.inputformat.class", + serConf.get().getClass("mapreduce.job.inputformat.class", InputFormat.class), - serConf.getHadoopConfiguration().getClass("value.class", Object.class)); + serConf.get().getClass("value.class", Object.class)); thrown.expect(IllegalArgumentException.class); thrown.expectMessage(expectedMessage); read.validateTransform(); @@ -387,7 +387,7 @@ public String apply(LongWritable input) { @Test public void testReadingData() throws Exception { HadoopInputFormatIO.Read read = HadoopInputFormatIO.read() - .withConfiguration(serConf.getHadoopConfiguration()); + .withConfiguration(serConf.get()); List> expected = TestEmployeeDataSet.getEmployeeData(); PCollection> actual = p.apply("ReadTest", read); PAssert.that(actual).containsInAnyOrder(expected); @@ -413,11 +413,11 @@ public void testReadDisplayData() { assertThat( displayData, hasDisplayItem("mapreduce.job.inputformat.class", - serConf.getHadoopConfiguration().get("mapreduce.job.inputformat.class"))); + serConf.get().get("mapreduce.job.inputformat.class"))); assertThat(displayData, - hasDisplayItem("key.class", serConf.getHadoopConfiguration().get("key.class"))); + hasDisplayItem("key.class", serConf.get().get("key.class"))); assertThat(displayData, - hasDisplayItem("value.class", serConf.getHadoopConfiguration().get("value.class"))); + hasDisplayItem("value.class", serConf.get().get("value.class"))); } /** From 185deebaa52bbf34592a21d86f316b4204fa09ba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Sun, 28 May 2017 11:38:08 +0200 Subject: [PATCH 126/159] Make SerializableConfiguration more robust by using Hadoop based serialization --- .../io/hadoop/SerializableConfiguration.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java index 8101f4b7f933..33c660a0ab1d 100644 --- a/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java +++ b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java @@ -49,21 +49,21 @@ public Configuration get() { return conf; } + @Override public void writeExternal(ObjectOutput out) throws IOException { - out.writeInt(conf.size()); - for (Map.Entry entry : conf) { - out.writeUTF(entry.getKey()); - out.writeUTF(entry.getValue()); - } + out.writeUTF(conf.getClass().getCanonicalName()); + conf.write(out); } @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - conf = new Configuration(false); - int size = in.readInt(); - for (int i = 0; i < size; i++) { - conf.set(in.readUTF(), in.readUTF()); + String className = in.readUTF(); + try { + conf = (Configuration) Class.forName(className).newInstance(); + conf.readFields(in); + } catch (InstantiationException | IllegalAccessException e) { + throw new IOException("Unable to create configuration: " + e); } } From 11bf82537a000f74b9690598cc5fb8e9173904d4 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 26 May 2017 16:39:13 -0700 Subject: [PATCH 127/159] [TRIVIAL] InstantCoder: stop boxing Longs unnecessarily In encode, and similar in decode, the existing path goes Instant->long (inside of converter) long->Long (returned from converter) Long->long (inside of LongCoder). This is a relatively small improvement, but as we encode timestamps for every single element, this is likely to make a difference in lightweight stages of pipelines. --- .../apache/beam/sdk/coders/InstantCoder.java | 81 +++++++++---------- 1 file changed, 38 insertions(+), 43 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java index 648493ea2ddc..e4fadef0631b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java @@ -17,11 +17,13 @@ */ package org.apache.beam.sdk.coders; -import com.google.common.base.Converter; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import java.io.UTFDataFormatException; import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Instant; @@ -39,53 +41,46 @@ public static InstantCoder of() { private static final InstantCoder INSTANCE = new InstantCoder(); private static final TypeDescriptor TYPE_DESCRIPTOR = new TypeDescriptor() {}; - private static final BigEndianLongCoder LONG_CODER = BigEndianLongCoder.of(); - private InstantCoder() {} - private static final Converter ORDER_PRESERVING_CONVERTER = - new LexicographicLongConverter(); - - /** - * Converts {@link Instant} to a {@code Long} representing its millis-since-epoch, - * but shifted so that the byte representation of negative values are lexicographically - * ordered before the byte representation of positive values. - * - *

This deliberately utilizes the well-defined overflow for {@code Long} values. - * See http://docs.oracle.com/javase/specs/jls/se7/html/jls-15.html#jls-15.18.2 - */ - private static class LexicographicLongConverter extends Converter { - - @Override - protected Long doForward(Instant instant) { - return instant.getMillis() - Long.MIN_VALUE; - } - - @Override - protected Instant doBackward(Long shiftedMillis) { - return new Instant(shiftedMillis + Long.MIN_VALUE); - } - } - @Override - public void encode(Instant value, OutputStream outStream) - throws CoderException, IOException { + public void encode(Instant value, OutputStream outStream) throws CoderException, IOException { if (value == null) { throw new CoderException("cannot encode a null Instant"); } - LONG_CODER.encode(ORDER_PRESERVING_CONVERTER.convert(value), outStream); + + // Converts {@link Instant} to a {@code long} representing its millis-since-epoch, + // but shifted so that the byte representation of negative values are lexicographically + // ordered before the byte representation of positive values. + // + // This deliberately utilizes the well-defined underflow for {@code long} values. + // See http://docs.oracle.com/javase/specs/jls/se7/html/jls-15.html#jls-15.18.2 + long shiftedMillis = value.getMillis() - Long.MIN_VALUE; + new DataOutputStream(outStream).writeLong(shiftedMillis); } @Override - public Instant decode(InputStream inStream) - throws CoderException, IOException { - return ORDER_PRESERVING_CONVERTER.reverse().convert(LONG_CODER.decode(inStream)); + public Instant decode(InputStream inStream) throws CoderException, IOException { + long shiftedMillis; + try { + shiftedMillis = new DataInputStream(inStream).readLong(); + } catch (EOFException | UTFDataFormatException exn) { + // These exceptions correspond to decoding problems, so change + // what kind of exception they're branded as. + throw new CoderException(exn); + } + + // Produces an {@link Instant} from a {@code long} representing its millis-since-epoch, + // but shifted so that the byte representation of negative values are lexicographically + // ordered before the byte representation of positive values. + // + // This deliberately utilizes the well-defined overflow for {@code long} values. + // See http://docs.oracle.com/javase/specs/jls/se7/html/jls-15.html#jls-15.18.2 + return new Instant(shiftedMillis + Long.MIN_VALUE); } @Override - public void verifyDeterministic() { - LONG_CODER.verifyDeterministic(); - } + public void verifyDeterministic() {} /** * {@inheritDoc} @@ -104,15 +99,15 @@ public boolean consistentWithEquals() { */ @Override public boolean isRegisterByteSizeObserverCheap(Instant value) { - return LONG_CODER.isRegisterByteSizeObserverCheap( - ORDER_PRESERVING_CONVERTER.convert(value)); + return true; } @Override - public void registerByteSizeObserver( - Instant value, ElementByteSizeObserver observer) throws Exception { - LONG_CODER.registerByteSizeObserver( - ORDER_PRESERVING_CONVERTER.convert(value), observer); + protected long getEncodedElementByteSize(Instant value) throws Exception { + if (value == null) { + throw new CoderException("cannot encode a null Instant"); + } + return 8; } @Override From 6a78bd3f09c99f49c5f27d15b3791f200bf5d53d Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Wed, 31 May 2017 09:38:02 -0700 Subject: [PATCH 128/159] Reduce Prevalence of PValue in the DirectRunner Use PCollection or PCollectionView explicitly. Retrieve views from the WriteView transform rather than visiting the view as an output PValue. --- .../beam/runners/direct/DirectGraph.java | 26 +++++++++++-------- .../runners/direct/DirectGraphVisitor.java | 25 ++++++++++-------- .../beam/runners/direct/DirectRunner.java | 4 ++- .../runners/direct/EvaluationContext.java | 17 +++++++++++- .../beam/runners/direct/WatermarkManager.java | 19 +++++++++++--- .../direct/DirectGraphVisitorTest.java | 3 +++ .../beam/runners/direct/DirectGraphs.java | 10 ++++++- .../runners/direct/EvaluationContextTest.java | 6 ++++- 8 files changed, 80 insertions(+), 30 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java index 83b214aca89a..c2c0afa73050 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java @@ -24,9 +24,9 @@ import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; /** @@ -34,39 +34,43 @@ * executed with the {@link DirectRunner}. */ class DirectGraph { - private final Map> producers; + private final Map, AppliedPTransform> producers; + private final Map, AppliedPTransform> viewWriters; private final ListMultimap> primitiveConsumers; - private final Set> views; private final Set> rootTransforms; private final Map, String> stepNames; public static DirectGraph create( - Map> producers, + Map, AppliedPTransform> producers, + Map, AppliedPTransform> viewWriters, ListMultimap> primitiveConsumers, - Set> views, Set> rootTransforms, Map, String> stepNames) { - return new DirectGraph(producers, primitiveConsumers, views, rootTransforms, stepNames); + return new DirectGraph(producers, viewWriters, primitiveConsumers, rootTransforms, stepNames); } private DirectGraph( - Map> producers, + Map, AppliedPTransform> producers, + Map, AppliedPTransform> viewWriters, ListMultimap> primitiveConsumers, - Set> views, Set> rootTransforms, Map, String> stepNames) { this.producers = producers; + this.viewWriters = viewWriters; this.primitiveConsumers = primitiveConsumers; - this.views = views; this.rootTransforms = rootTransforms; this.stepNames = stepNames; } - AppliedPTransform getProducer(PValue produced) { + AppliedPTransform getProducer(PCollection produced) { return producers.get(produced); } + AppliedPTransform getWriter(PCollectionView view) { + return viewWriters.get(view); + } + List> getPrimitiveConsumers(PValue consumed) { return primitiveConsumers.get(consumed); } @@ -76,7 +80,7 @@ private DirectGraph( } Set> getViews() { - return views; + return viewWriters.keySet(); } String getStepName(AppliedPTransform step) { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java index 01204e3049dd..d54de5d9b5ee 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java @@ -30,9 +30,9 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; /** @@ -42,12 +42,12 @@ */ class DirectGraphVisitor extends PipelineVisitor.Defaults { - private Map> producers = new HashMap<>(); + private Map, AppliedPTransform> producers = new HashMap<>(); + private Map, AppliedPTransform> viewWriters = new HashMap<>(); private ListMultimap> primitiveConsumers = ArrayListMultimap.create(); - private Set> views = new HashSet<>(); private Set> rootTransforms = new HashSet<>(); private Map, String> stepNames = new HashMap<>(); private int numTransforms = 0; @@ -86,17 +86,19 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { for (PValue value : node.getInputs().values()) { primitiveConsumers.put(value, appliedTransform); } + if (node.getTransform() instanceof ViewOverrideFactory.WriteView) { + viewWriters.put( + ((ViewOverrideFactory.WriteView) node.getTransform()).getView(), + node.toAppliedPTransform(getPipeline())); + } } } @Override public void visitValue(PValue value, TransformHierarchy.Node producer) { AppliedPTransform appliedTransform = getAppliedTransform(producer); - if (value instanceof PCollectionView) { - views.add((PCollectionView) value); - } - if (!producers.containsKey(value)) { - producers.put(value, appliedTransform); + if (value instanceof PCollection && !producers.containsKey(value)) { + producers.put((PCollection) value, appliedTransform); } } @@ -111,11 +113,12 @@ private String genStepName() { } /** - * Get the graph constructed by this {@link DirectGraphVisitor}, which provides - * lookups for producers and consumers of {@link PValue PValues}. + * Get the graph constructed by this {@link DirectGraphVisitor}, which provides lookups for + * producers and consumers of {@link PValue PValues}. */ public DirectGraph getGraph() { checkState(finalized, "Can't get a graph before the Pipeline has been completely traversed"); - return DirectGraph.create(producers, primitiveConsumers, views, rootTransforms, stepNames); + return DirectGraph.create( + producers, viewWriters, primitiveConsumers, rootTransforms, stepNames); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java index 69dea8fa4cb6..dbd1ec47ed54 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.direct; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -220,7 +221,8 @@ public DirectPipelineResult run(Pipeline pipeline) { * iteration order based on the order at which elements are added to it. */ @SuppressWarnings("rawtypes") - private List defaultTransformOverrides() { + @VisibleForTesting + List defaultTransformOverrides() { return ImmutableList.builder() .add( PTransformOverride.of( diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java index 88ce85a7cd22..e215070dac07 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java @@ -276,7 +276,7 @@ public void add(Iterable> values) { * callback will be executed regardless of whether values have been produced. */ public void scheduleAfterOutputWouldBeProduced( - PValue value, + PCollection value, BoundedWindow window, WindowingStrategy windowingStrategy, Runnable runnable) { @@ -286,6 +286,21 @@ public void scheduleAfterOutputWouldBeProduced( fireAvailableCallbacks(producing); } + /** + * Schedule a callback to be executed after output would be produced for the given window if there + * had been input. + */ + public void scheduleAfterOutputWouldBeProduced( + PCollectionView view, + BoundedWindow window, + WindowingStrategy windowingStrategy, + Runnable runnable) { + AppliedPTransform producing = graph.getWriter(view); + callbackExecutor.callOnGuaranteedFiring(producing, window, windowingStrategy, runnable); + + fireAvailableCallbacks(producing); + } + /** * Schedule a callback to be executed after the given window is expired. * diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java index 4f1b8319dc2d..40ce163012fb 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java @@ -60,6 +60,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Instant; @@ -790,6 +791,18 @@ private WatermarkManager(Clock clock, DirectGraph graph) { } } + private TransformWatermarks getValueWatermark(PValue pvalue) { + if (pvalue instanceof PCollection) { + return getTransformWatermark(graph.getProducer((PCollection) pvalue)); + } else if (pvalue instanceof PCollectionView) { + return getTransformWatermark(graph.getWriter((PCollectionView) pvalue)); + } else { + throw new IllegalArgumentException( + String.format( + "Unknown type of %s %s", PValue.class.getSimpleName(), pvalue.getClass())); + } + } + private TransformWatermarks getTransformWatermark(AppliedPTransform transform) { TransformWatermarks wms = transformToWatermarks.get(transform); if (wms == null) { @@ -824,8 +837,7 @@ private Collection getInputProcessingWatermarks(AppliedPTransform getInputWatermarks(AppliedPTransform transform) inputWatermarksBuilder.add(THE_END_OF_TIME); } for (PValue pvalue : inputs.values()) { - Watermark producerOutputWatermark = - getTransformWatermark(graph.getProducer(pvalue)).outputWatermark; + Watermark producerOutputWatermark = getValueWatermark(pvalue).outputWatermark; inputWatermarksBuilder.add(producerOutputWatermark); } List inputCollectionWatermarks = inputWatermarksBuilder.build(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java index 7f46a0e4c9c3..576edf364fae 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphVisitorTest.java @@ -78,6 +78,9 @@ public void processElement(DoFn.ProcessContext c) .apply(View.asList()); PCollectionView singletonView = p.apply("singletonCreate", Create.of(1, 2, 3)).apply(View.asSingleton()); + p.replaceAll( + DirectRunner.fromOptions(TestPipeline.testingPipelineOptions()) + .defaultTransformOverrides()); p.traverseTopologically(visitor); assertThat( visitor.getGraph().getViews(), diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphs.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphs.java index 2f048fa7cab2..43de091b263d 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphs.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGraphs.java @@ -19,6 +19,8 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; /** Test utilities for the {@link DirectRunner}. */ @@ -30,6 +32,12 @@ public static DirectGraph getGraph(Pipeline p) { } public static AppliedPTransform getProducer(PValue value) { - return getGraph(value.getPipeline()).getProducer(value); + if (value instanceof PCollection) { + return getGraph(value.getPipeline()).getProducer((PCollection) value); + } else if (value instanceof PCollectionView) { + return getGraph(value.getPipeline()).getWriter((PCollectionView) value); + } + throw new IllegalArgumentException( + String.format("Unexpected type of %s %s", PValue.class.getSimpleName(), value.getClass())); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index 80b04f80a06a..c0e43d6c6468 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -101,6 +101,10 @@ public void setup() { view = created.apply(View.asIterable()); unbounded = p.apply(GenerateSequence.from(0)); + p.replaceAll( + DirectRunner.fromOptions(TestPipeline.testingPipelineOptions()) + .defaultTransformOverrides()); + KeyedPValueTrackingVisitor keyedPValueTrackingVisitor = KeyedPValueTrackingVisitor.create(); p.traverseTopologically(keyedPValueTrackingVisitor); @@ -116,7 +120,7 @@ public void setup() { createdProducer = graph.getProducer(created); downstreamProducer = graph.getProducer(downstream); - viewProducer = graph.getProducer(view); + viewProducer = graph.getWriter(view); unboundedProducer = graph.getProducer(unbounded); } From a0444b8ce7f6d39b039612190102e146ef4148dd Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Tue, 30 May 2017 16:12:23 -0700 Subject: [PATCH 129/159] Flink*DoFnFunction: fix check for single-output dofns Fixes Findbugs and (presumably) increases efficiency --- .../runners/flink/translation/functions/FlinkDoFnFunction.java | 2 +- .../flink/translation/functions/FlinkStatefulDoFnFunction.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index 42a8833fea7b..ab2ac6b0ea2f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -90,7 +90,7 @@ public void mapPartition( RuntimeContext runtimeContext = getRuntimeContext(); DoFnRunners.OutputManager outputManager; - if (outputMap == null) { + if (outputMap.size() == 1) { outputManager = new FlinkDoFnFunction.DoFnOutputManager(out); } else { // it has some additional outputs diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java index b07576893d01..11d4fee48957 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java @@ -91,7 +91,7 @@ public void reduce( RuntimeContext runtimeContext = getRuntimeContext(); DoFnRunners.OutputManager outputManager; - if (outputMap == null) { + if (outputMap.size() == 1) { outputManager = new FlinkDoFnFunction.DoFnOutputManager(out); } else { // it has some additional Outputs From 5780fc5e8cd64911d9612d89896b9d68be4f621f Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 31 May 2017 10:50:46 -0700 Subject: [PATCH 130/159] Add RawUnion code to FlinkDoFnFunction --- .../flink/translation/functions/FlinkDoFnFunction.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index ab2ac6b0ea2f..d8ed622ffb89 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -146,7 +146,9 @@ static class DoFnOutputManager @Override @SuppressWarnings("unchecked") public void output(TupleTag tag, WindowedValue output) { - collector.collect(output); + collector.collect( + WindowedValue.of(new RawUnionValue(0 /* single output */, output.getValue()), + output.getTimestamp(), output.getWindows(), output.getPane())); } } From a158fc178e1297f04f4f18975383ec1dc69bc0d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Wed, 10 May 2017 07:39:56 +0200 Subject: [PATCH 131/159] [BEAM-2246] Use CLIENT_ACK instead of AUTO_ACK in JmsIO --- .../org/apache/beam/sdk/io/jms/JmsIO.java | 5 +- .../org/apache/beam/sdk/io/jms/JmsIOTest.java | 78 +++++++++++++++++++ 2 files changed, 81 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java index b8355ad4a30f..c5e51508c3ac 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java @@ -379,7 +379,8 @@ public Coder getDefaultOutputCoder() { } - private static class UnboundedJmsReader extends UnboundedReader { + @VisibleForTesting + static class UnboundedJmsReader extends UnboundedReader { private UnboundedJmsSource source; private JmsCheckpointMark checkpointMark; @@ -421,7 +422,7 @@ public boolean start() throws IOException { } try { - this.session = this.connection.createSession(false, Session.AUTO_ACKNOWLEDGE); + this.session = this.connection.createSession(false, Session.CLIENT_ACKNOWLEDGE); } catch (Exception e) { throw new IOException("Error creating JMS session", e); } diff --git a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java index 7edda1acd296..43c050e3dac4 100644 --- a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java +++ b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java @@ -23,10 +23,12 @@ 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.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; +import java.util.Enumeration; import java.util.List; import javax.jms.Connection; import javax.jms.ConnectionFactory; @@ -34,6 +36,7 @@ import javax.jms.Message; import javax.jms.MessageConsumer; import javax.jms.MessageProducer; +import javax.jms.QueueBrowser; import javax.jms.Session; import javax.jms.TextMessage; import org.apache.activemq.ActiveMQConnectionFactory; @@ -71,6 +74,7 @@ public class JmsIOTest { private BrokerService broker; private ConnectionFactory connectionFactory; + private ConnectionFactory connectionFactoryWithoutPrefetch; @Rule public final transient TestPipeline pipeline = TestPipeline.create(); @@ -98,6 +102,8 @@ public void startBroker() throws Exception { // create JMS connection factory connectionFactory = new ActiveMQConnectionFactory(BROKER_URL); + connectionFactoryWithoutPrefetch = + new ActiveMQConnectionFactory(BROKER_URL + "?jms.prefetchPolicy.all=0"); } @After @@ -236,4 +242,76 @@ public void testSplitForTopic() throws Exception { assertEquals(1, splits.size()); } + @Test + public void testCheckpointMark() throws Exception { + // we are using no prefetch here + // prefetch is an ActiveMQ feature: to make efficient use of network resources the broker + // utilizes a 'push' model to dispatch messages to consumers. However, in the case of our + // test, it means that we can have some latency between the receiveNoWait() method used by + // the consumer and the prefetch buffer populated by the broker. Using a prefetch to 0 means + // that the consumer will poll for message, which is exactly what we want for the test. + Connection connection = connectionFactoryWithoutPrefetch.createConnection(USERNAME, PASSWORD); + connection.start(); + Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE); + MessageProducer producer = session.createProducer(session.createQueue(QUEUE)); + for (int i = 0; i < 10; i++) { + producer.send(session.createTextMessage("test " + i)); + } + producer.close(); + session.close(); + connection.close(); + + JmsIO.Read spec = JmsIO.read() + .withConnectionFactory(connectionFactoryWithoutPrefetch) + .withUsername(USERNAME) + .withPassword(PASSWORD) + .withQueue(QUEUE); + JmsIO.UnboundedJmsSource source = new JmsIO.UnboundedJmsSource(spec); + JmsIO.UnboundedJmsReader reader = source.createReader(null, null); + + // start the reader and move to the first record + assertTrue(reader.start()); + + // consume 3 messages (NB: start already consumed the first message) + for (int i = 0; i < 3; i++) { + assertTrue(reader.advance()); + } + + // the messages are still pending in the queue (no ACK yet) + assertEquals(10, count(QUEUE)); + + // we finalize the checkpoint + reader.getCheckpointMark().finalizeCheckpoint(); + + // the checkpoint finalize ack the messages, and so they are not pending in the queue anymore + assertEquals(6, count(QUEUE)); + + // we read the 6 pending messages + for (int i = 0; i < 6; i++) { + assertTrue(reader.advance()); + } + + // still 6 pending messages as we didn't finalize the checkpoint + assertEquals(6, count(QUEUE)); + + // we finalize the checkpoint: no more message in the queue + reader.getCheckpointMark().finalizeCheckpoint(); + + assertEquals(0, count(QUEUE)); + } + + private int count(String queue) throws Exception { + Connection connection = connectionFactory.createConnection(USERNAME, PASSWORD); + connection.start(); + Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE); + QueueBrowser browser = session.createBrowser(session.createQueue(queue)); + Enumeration messages = browser.getEnumeration(); + int count = 0; + while (messages.hasMoreElements()) { + messages.nextElement(); + count++; + } + return count; + } + } From 2ee7d69ea6f130186b38ae21abc2ab06aee2c5a7 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 26 May 2017 16:06:23 -0700 Subject: [PATCH 132/159] Add WriteFilesPayload to Runner API well-known transforms --- .../runner-api/src/main/proto/beam_runner_api.proto | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto index 16122093c812..acc4a357bb83 100644 --- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto +++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto @@ -325,6 +325,17 @@ message CombinePayload { map side_inputs = 4; } +// The payload for the special-but-not-primitive WriteFiles transform. +message WriteFilesPayload { + + // (Required) The SdkFunctionSpec of the FileBasedSink. + SdkFunctionSpec sink = 1; + + bool windowed_writes = 2; + + bool runner_determined_sharding = 3; +} + // A coder, the binary format for serialization and deserialization of data in // a pipeline. message Coder { From 0093cf5e53cbe6f3e6d61f91ea44d0d1b0f0f901 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 26 May 2017 16:06:57 -0700 Subject: [PATCH 133/159] Make WriteFiles config publicly readable --- .../src/main/java/org/apache/beam/sdk/io/WriteFiles.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java index 0786e5d72fe9..2fd10ace994a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java @@ -97,7 +97,7 @@ public class WriteFiles extends PTransform, PDone> { // when a pipeline is updated. @Nullable private final ValueProvider numShardsProvider; - private boolean windowedWrites; + private final boolean windowedWrites; /** * Creates a {@link WriteFiles} transform that writes to the given {@link FileBasedSink}, letting @@ -166,6 +166,13 @@ public FileBasedSink getSink() { return sink; } + /** + * Returns whether or not to perform windowed writes. + */ + public boolean isWindowedWrites() { + return windowedWrites; + } + /** * Gets the {@link PTransform} that will be used to determine sharding. This can be either a * static number of shards (as following a call to {@link #withNumShards(int)}), dynamic (by From 5cf5a1b2ba41b1c0c0961eb535e6ba220f61435f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Sun, 28 May 2017 11:46:03 +0200 Subject: [PATCH 134/159] Rename class Logger object names to the standard convention 'LOG' --- .../apache/beam/sdk/transforms/GroupIntoBatches.java | 12 ++++++------ .../java/org/apache/beam/sdk/util/ClassPath.java | 8 ++++---- .../beam/sdk/transforms/GroupIntoBatchesTest.java | 4 ++-- .../beam/sdk/io/elasticsearch/ElasticsearchIOIT.java | 4 ++-- .../io/hadoop/inputformat/HIFIOWithElasticTest.java | 10 +++++----- 5 files changed, 19 insertions(+), 19 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java index b02336339ffa..a79b07bab128 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java @@ -99,7 +99,7 @@ public PCollection>> expand(PCollection> in static class GroupIntoBatchesDoFn extends DoFn, KV>> { - private static final Logger LOGGER = LoggerFactory.getLogger(GroupIntoBatchesDoFn.class); + private static final Logger LOG = LoggerFactory.getLogger(GroupIntoBatchesDoFn.class); private static final String END_OF_WINDOW_ID = "endOFWindow"; private static final String BATCH_ID = "batch"; private static final String NUM_ELEMENTS_IN_BATCH_ID = "numElementsInBatch"; @@ -160,13 +160,13 @@ public void processElement( BoundedWindow window) { Instant windowExpires = window.maxTimestamp().plus(allowedLateness); - LOGGER.debug( + LOG.debug( "*** SET TIMER *** to point in time {} for window {}", windowExpires.toString(), window.toString()); timer.set(windowExpires); key.write(c.element().getKey()); batch.add(c.element().getValue()); - LOGGER.debug("*** BATCH *** Add element for window {} ", window.toString()); + LOG.debug("*** BATCH *** Add element for window {} ", window.toString()); // blind add is supported with combiningState numElementsInBatch.add(1L); Long num = numElementsInBatch.read(); @@ -175,7 +175,7 @@ public void processElement( batch.readLater(); } if (num >= batchSize) { - LOGGER.debug("*** END OF BATCH *** for window {}", window.toString()); + LOG.debug("*** END OF BATCH *** for window {}", window.toString()); flushBatch(c, key, batch, numElementsInBatch); } } @@ -188,7 +188,7 @@ public void onTimerCallback( @StateId(NUM_ELEMENTS_IN_BATCH_ID) CombiningState numElementsInBatch, BoundedWindow window) { - LOGGER.debug( + LOG.debug( "*** END OF WINDOW *** for timer timestamp {} in windows {}", context.timestamp(), window.toString()); flushBatch(context, key, batch, numElementsInBatch); @@ -205,7 +205,7 @@ private void flushBatch( c.output(KV.of(key.read(), values)); } batch.clear(); - LOGGER.debug("*** BATCH *** clear"); + LOG.debug("*** BATCH *** clear"); numElementsInBatch.clear(); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ClassPath.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ClassPath.java index 271bce0fed58..2f9e0493543e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ClassPath.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ClassPath.java @@ -75,7 +75,7 @@ @Beta final class ClassPath { - private static final Logger logger = LoggerFactory.getLogger(ClassPath.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger(ClassPath.class.getName()); private static final Predicate IS_TOP_LEVEL = new Predicate() { @@ -374,7 +374,7 @@ private void scanFrom(File file, ClassLoader classloader) throws IOException { return; } } catch (SecurityException e) { - logger.warn("Cannot access " + file + ": " + e); + LOG.warn("Cannot access " + file + ": " + e); return; } if (file.isDirectory()) { @@ -429,7 +429,7 @@ static ImmutableSet getClassPathFromManifest(File jarFile, url = getClassPathEntry(jarFile, path); } catch (MalformedURLException e) { // Ignore bad entry - logger.warn("Invalid Class-Path entry: " + path); + LOG.warn("Invalid Class-Path entry: " + path); continue; } if (url.getProtocol().equals("file")) { @@ -509,7 +509,7 @@ private void scanDirectory(File directory, ClassLoader classloader, String packa throws IOException { File[] files = directory.listFiles(); if (files == null) { - logger.warn("Cannot read directory " + directory); + LOG.warn("Cannot read directory " + directory); // IO error, just skip the directory return; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java index 54e2d5aa0727..c213d6adbef2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java @@ -56,7 +56,7 @@ public class GroupIntoBatchesTest implements Serializable { private static final int BATCH_SIZE = 5; private static final long NUM_ELEMENTS = 10; private static final int ALLOWED_LATENESS = 0; - private static final Logger LOGGER = LoggerFactory.getLogger(GroupIntoBatchesTest.class); + private static final Logger LOG = LoggerFactory.getLogger(GroupIntoBatchesTest.class); @Rule public transient TestPipeline pipeline = TestPipeline.create(); private transient ArrayList> data = createTestData(); @@ -159,7 +159,7 @@ public void testInStreamingMode() { new DoFn, Void>() { @ProcessElement public void processElement(ProcessContext c, BoundedWindow window) { - LOGGER.debug( + LOG.debug( "*** ELEMENT: ({},{}) *** with timestamp %s in window %s", c.element().getKey(), c.element().getValue(), diff --git a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java index d968bc2f42ef..2d6393adc56f 100644 --- a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java +++ b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java @@ -56,7 +56,7 @@ * */ public class ElasticsearchIOIT { - private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchIOIT.class); + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchIOIT.class); private static TransportClient client; private static IOTestPipelineOptions options; private static ElasticsearchIO.ConnectionConfiguration readConnectionConfiguration; @@ -142,7 +142,7 @@ public void testEstimatedSizesVolume() throws Exception { // can't use equal assert as Elasticsearch indexes never have same size // (due to internal Elasticsearch implementation) long estimatedSize = initialSource.getEstimatedSizeBytes(options); - LOGGER.info("Estimated size: {}", estimatedSize); + LOG.info("Estimated size: {}", estimatedSize); assertThat( "Wrong estimated size bellow minimum", estimatedSize, diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithElasticTest.java b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithElasticTest.java index 99d371df4f55..8745521a3064 100644 --- a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithElasticTest.java +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithElasticTest.java @@ -74,7 +74,7 @@ public class HIFIOWithElasticTest implements Serializable { private static final long serialVersionUID = 1L; - private static final Logger LOGGER = LoggerFactory.getLogger(HIFIOWithElasticTest.class); + private static final Logger LOG = LoggerFactory.getLogger(HIFIOWithElasticTest.class); private static final String ELASTIC_IN_MEM_HOSTNAME = "127.0.0.1"; private static final String ELASTIC_IN_MEM_PORT = "9200"; private static final String ELASTIC_INTERNAL_VERSION = "5.x"; @@ -217,9 +217,9 @@ public static void startElasticEmbeddedServer() .put("node.ingest", TRUE).build(); node = new PluginNode(settings); node.start(); - LOGGER.info("Elastic in memory server started."); + LOG.info("Elastic in memory server started."); prepareElasticIndex(); - LOGGER.info("Prepared index " + ELASTIC_INDEX_NAME + LOG.info("Prepared index " + ELASTIC_INDEX_NAME + "and populated data on elastic in memory server."); } @@ -243,9 +243,9 @@ private static void prepareElasticIndex() throws InterruptedException { public static void shutdown() throws IOException { DeleteIndexRequest indexRequest = new DeleteIndexRequest(ELASTIC_INDEX_NAME); node.client().admin().indices().delete(indexRequest).actionGet(); - LOGGER.info("Deleted index " + ELASTIC_INDEX_NAME + " from elastic in memory server"); + LOG.info("Deleted index " + ELASTIC_INDEX_NAME + " from elastic in memory server"); node.close(); - LOGGER.info("Closed elastic in memory server node."); + LOG.info("Closed elastic in memory server node."); deleteElasticDataDirectory(); } From be3c39f06521f9c5c5500be66688fc17e3680e25 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 30 May 2017 14:16:22 -0700 Subject: [PATCH 135/159] Rephrases messages on RetryHttpRequestInitializer "will NOT retry" messages are confusing some customers into thinking that an entire high-level operation inside a Beam job, involving this request as an implementation detail, will not be retried (e.g. a BigQuery import etc.) In reality there's many levels of retries and RetryHttpRequestInitializer can not be aware of them. Retrying at a higher level may or may not happen, and it's up to higher-level components to log that (they usually do). --- .../beam/sdk/util/RetryHttpRequestInitializer.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java index 2b7135e94cd9..e5b48d39664f 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java @@ -73,7 +73,9 @@ public boolean handleIOException(HttpRequest request, boolean supportsRetry) if (willRetry) { LOG.debug("Request failed with IOException, will retry: {}", request.getUrl()); } else { - LOG.warn("Request failed with IOException, will NOT retry: {}", request.getUrl()); + LOG.warn( + "Request failed with IOException (caller responsible for retrying): {}", + request.getUrl()); } return willRetry; } @@ -105,12 +107,14 @@ public boolean handleResponse(HttpRequest request, HttpResponse response, boolean supportsRetry) throws IOException { boolean retry = handler.handleResponse(request, response, supportsRetry); if (retry) { - LOG.debug("Request failed with code {} will retry: {}", + LOG.debug("Request failed with code {}, will retry: {}", response.getStatusCode(), request.getUrl()); } else if (!ignoredResponseCodes.contains(response.getStatusCode())) { - LOG.warn("Request failed with code {}, will NOT retry: {}", - response.getStatusCode(), request.getUrl()); + LOG.warn( + "Request failed with code {} (caller responsible for retrying): {}", + response.getStatusCode(), + request.getUrl()); } return retry; From fdf7e330743d7e62a25f9e7d2d5ea219843a87f1 Mon Sep 17 00:00:00 2001 From: Maria Garcia Herrero Date: Thu, 25 May 2017 18:21:46 -0700 Subject: [PATCH 136/159] Add template examples to snippets.py --- .../apache_beam/examples/snippets/snippets.py | 86 +++++++++++++++++++ .../examples/snippets/snippets_test.py | 14 ++- 2 files changed, 99 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py index 70929e9e8fae..3a5f9b15a456 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets.py +++ b/sdks/python/apache_beam/examples/snippets/snippets.py @@ -502,6 +502,51 @@ def process(self, element): p.visit(SnippetUtils.RenameFiles(renames)) +def examples_wordcount_templated(renames): + """Templated WordCount example snippet.""" + import re + + import apache_beam as beam + from apache_beam.io import ReadFromText + from apache_beam.io import WriteToText + from apache_beam.options.pipeline_options import PipelineOptions + + # [START example_wordcount_templated] + class WordcountTemplatedOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + # Use add_value_provider_argument for arguments to be templatable + # Use add_argument as usual for non-templatable arguments + parser.add_value_provider_argument( + '--input', + help='Path of the file to read from') + parser.add_argument( + '--output', + required=True, + help='Output file to write results to.') + pipeline_options = PipelineOptions(['--output', 'some/output_path']) + p = beam.Pipeline(options=pipeline_options) + + wordcount_options = pipeline_options.view_as(WordcountTemplatedOptions) + lines = p | 'Read' >> ReadFromText(wordcount_options.input) + # [END example_wordcount_templated] + + ( + lines + | 'ExtractWords' >> beam.FlatMap( + lambda x: re.findall(r'[A-Za-z\']+', x)) + | 'PairWithOnes' >> beam.Map(lambda x: (x, 1)) + | 'Group' >> beam.GroupByKey() + | 'Sum' >> beam.Map(lambda (word, ones): (word, sum(ones))) + | 'Format' >> beam.Map(lambda (word, c): '%s: %s' % (word, c)) + | 'Write' >> WriteToText(wordcount_options.output) + ) + + p.visit(SnippetUtils.RenameFiles(renames)) + result = p.run() + result.wait_until_finish() + + def examples_wordcount_debugging(renames): """DebuggingWordCount example snippets.""" import re @@ -569,6 +614,47 @@ def process(self, element): p.visit(SnippetUtils.RenameFiles(renames)) +def examples_ptransforms_templated(renames): + # [START examples_ptransforms_templated] + import apache_beam as beam + from apache_beam.io import WriteToText + from apache_beam.options.pipeline_options import PipelineOptions + from apache_beam.options.value_provider import StaticValueProvider + + class TemplatedUserOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_value_provider_argument('--templated_int', type=int) + + class MySumFn(beam.DoFn): + def __init__(self, templated_int): + self.templated_int = templated_int + + def process(self, an_int): + yield self.templated_int.get() + an_int + + pipeline_options = PipelineOptions() + p = beam.Pipeline(options=pipeline_options) + + user_options = pipeline_options.view_as(TemplatedUserOptions) + my_sum_fn = MySumFn(user_options.templated_int) + sum = (p + | 'ReadCollection' >> beam.io.ReadFromText( + 'gs://some/integer_collection') + | 'StringToInt' >> beam.Map(lambda w: int(w)) + | 'AddGivenInt' >> beam.ParDo(my_sum_fn) + | 'WriteResultingCollection' >> WriteToText('some/output_path')) + # [END examples_ptransforms_templated] + + # Templates are not supported by DirectRunner (only by DataflowRunner) + # so a value must be provided at graph-construction time + my_sum_fn.templated_int = StaticValueProvider(int, 10) + + p.visit(SnippetUtils.RenameFiles(renames)) + result = p.run() + result.wait_until_finish() + + import apache_beam as beam from apache_beam.io import iobase from apache_beam.io.range_trackers import OffsetRangeTracker diff --git a/sdks/python/apache_beam/examples/snippets/snippets_test.py b/sdks/python/apache_beam/examples/snippets/snippets_test.py index e302465985a4..9183d0dfea19 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets_test.py +++ b/sdks/python/apache_beam/examples/snippets/snippets_test.py @@ -636,7 +636,8 @@ def test_pipeline_logging(self): def test_examples_wordcount(self): pipelines = [snippets.examples_wordcount_minimal, snippets.examples_wordcount_wordcount, - snippets.pipeline_monitoring] + snippets.pipeline_monitoring, + snippets.examples_wordcount_templated] for pipeline in pipelines: temp_path = self.create_temp_file( @@ -647,6 +648,17 @@ def test_examples_wordcount(self): self.get_output(result_path), ['abc: 2', 'def: 1', 'ghi: 1', 'jkl: 1']) + def test_examples_ptransforms_templated(self): + pipelines = [snippets.examples_ptransforms_templated] + + for pipeline in pipelines: + temp_path = self.create_temp_file('1\n 2\n 3') + result_path = self.create_temp_file() + pipeline({'read': temp_path, 'write': result_path}) + self.assertEqual( + self.get_output(result_path), + ['11', '12', '13']) + def test_examples_wordcount_debugging(self): temp_path = self.create_temp_file( 'Flourish Flourish Flourish stomach abc def') From f3ed5a4f4f05c67371b51e6f8742f554b282eedf Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 30 May 2017 14:43:25 -0700 Subject: [PATCH 137/159] Add WriteFiles translation --- .../construction/PTransformTranslation.java | 3 + .../construction/WriteFilesTranslation.java | 152 ++++++++++++++ .../WriteFilesTranslationTest.java | 186 ++++++++++++++++++ 3 files changed, 341 insertions(+) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index 00ea55e6e1ed..99d1e85025a7 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -50,6 +50,9 @@ public class PTransformTranslation { public static final String READ_TRANSFORM_URN = "urn:beam:transform:read:v1"; public static final String WINDOW_TRANSFORM_URN = "urn:beam:transform:window:v1"; + // Less well-known. And where shall these live? + public static final String WRITE_FILES_TRANSFORM_URN = "urn:beam:transform:write_files:0.1"; + private static final Map, TransformPayloadTranslator> KNOWN_PAYLOAD_TRANSLATORS = loadTransformPayloadTranslators(); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java new file mode 100644 index 000000000000..99b77efd9d47 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.core.construction; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.WriteFilesPayload; +import org.apache.beam.sdk.io.FileBasedSink; +import org.apache.beam.sdk.io.WriteFiles; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; + +/** + * Utility methods for translating a {@link WriteFiles} to and from {@link RunnerApi} + * representations. + */ +public class WriteFilesTranslation { + + /** The URN for an unknown Java {@link FileBasedSink}. */ + public static final String CUSTOM_JAVA_FILE_BASED_SINK_URN = + "urn:beam:file_based_sink:javasdk:0.1"; + + @VisibleForTesting + static WriteFilesPayload toProto(WriteFiles transform) { + return WriteFilesPayload.newBuilder() + .setSink(toProto(transform.getSink())) + .setWindowedWrites(transform.isWindowedWrites()) + .setRunnerDeterminedSharding( + transform.getNumShards() == null && transform.getSharding() == null) + .build(); + } + + private static SdkFunctionSpec toProto(FileBasedSink sink) { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(CUSTOM_JAVA_FILE_BASED_SINK_URN) + .setParameter( + Any.pack( + BytesValue.newBuilder() + .setValue( + ByteString.copyFrom(SerializableUtils.serializeToByteArray(sink))) + .build()))) + .build(); + } + + @VisibleForTesting + static FileBasedSink sinkFromProto(SdkFunctionSpec sinkProto) throws IOException { + checkArgument( + sinkProto.getSpec().getUrn().equals(CUSTOM_JAVA_FILE_BASED_SINK_URN), + "Cannot extract %s instance from %s with URN %s", + FileBasedSink.class.getSimpleName(), + FunctionSpec.class.getSimpleName(), + sinkProto.getSpec().getUrn()); + + byte[] serializedSink = + sinkProto.getSpec().getParameter().unpack(BytesValue.class).getValue().toByteArray(); + + return (FileBasedSink) + SerializableUtils.deserializeFromByteArray( + serializedSink, FileBasedSink.class.getSimpleName()); + } + + public static FileBasedSink getSink( + AppliedPTransform, PDone, ? extends PTransform, PDone>> + transform) + throws IOException { + return (FileBasedSink) sinkFromProto(getWriteFilesPayload(transform).getSink()); + } + + public static boolean isWindowedWrites( + AppliedPTransform, PDone, ? extends PTransform, PDone>> + transform) + throws IOException { + return getWriteFilesPayload(transform).getWindowedWrites(); + } + + public static boolean isRunnerDeterminedSharding( + AppliedPTransform, PDone, ? extends PTransform, PDone>> + transform) + throws IOException { + return getWriteFilesPayload(transform).getRunnerDeterminedSharding(); + } + + private static WriteFilesPayload getWriteFilesPayload( + AppliedPTransform, PDone, ? extends PTransform, PDone>> + transform) + throws IOException { + return PTransformTranslation.toProto( + transform, Collections.>emptyList(), SdkComponents.create()) + .getSpec() + .getParameter() + .unpack(WriteFilesPayload.class); + } + + static class WriteFilesTranslator implements TransformPayloadTranslator> { + @Override + public String getUrn(WriteFiles transform) { + return PTransformTranslation.WRITE_FILES_TRANSFORM_URN; + } + + @Override + public FunctionSpec translate( + AppliedPTransform> transform, SdkComponents components) { + return FunctionSpec.newBuilder() + .setUrn(getUrn(transform.getTransform())) + .setParameter(Any.pack(toProto(transform.getTransform()))) + .build(); + } + } + + /** Registers {@link WriteFilesTranslator}. */ + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class Registrar implements TransformPayloadTranslatorRegistrar { + @Override + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return Collections.singletonMap(WriteFiles.class, new WriteFilesTranslator()); + } + } +} diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java new file mode 100644 index 000000000000..739034cfa860 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.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.core.construction; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import com.google.common.collect.ImmutableList; +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.ParDoPayload; +import org.apache.beam.sdk.io.FileBasedSink; +import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.WriteFiles; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; +import org.junit.runners.Suite; + +/** Tests for {@link WriteFilesTranslation}. */ +@RunWith(Suite.class) +@Suite.SuiteClasses({ + WriteFilesTranslationTest.TestWriteFilesPayloadTranslation.class, +}) +public class WriteFilesTranslationTest { + + /** Tests for translating various {@link ParDo} transforms to/from {@link ParDoPayload} protos. */ + @RunWith(Parameterized.class) + public static class TestWriteFilesPayloadTranslation { + @Parameters(name = "{index}: {0}") + public static Iterable> data() { + return ImmutableList.>of( + WriteFiles.to(new DummySink()), + WriteFiles.to(new DummySink()).withWindowedWrites(), + WriteFiles.to(new DummySink()).withNumShards(17), + WriteFiles.to(new DummySink()).withWindowedWrites().withNumShards(42)); + } + + @Parameter(0) + public WriteFiles writeFiles; + + public static TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); + + @Test + public void testEncodedProto() throws Exception { + RunnerApi.WriteFilesPayload payload = WriteFilesTranslation.toProto(writeFiles); + + assertThat( + payload.getRunnerDeterminedSharding(), + equalTo(writeFiles.getNumShards() == null && writeFiles.getSharding() == null)); + + assertThat(payload.getWindowedWrites(), equalTo(writeFiles.isWindowedWrites())); + + assertThat( + (FileBasedSink) WriteFilesTranslation.sinkFromProto(payload.getSink()), + equalTo(writeFiles.getSink())); + } + + @Test + public void testExtractionDirectFromTransform() throws Exception { + PCollection input = p.apply(Create.of("hello")); + PDone output = input.apply(writeFiles); + + AppliedPTransform, PDone, WriteFiles> appliedPTransform = + AppliedPTransform., PDone, WriteFiles>of( + "foo", input.expand(), output.expand(), writeFiles, p); + + assertThat( + WriteFilesTranslation.isRunnerDeterminedSharding(appliedPTransform), + equalTo(writeFiles.getNumShards() == null && writeFiles.getSharding() == null)); + + assertThat( + WriteFilesTranslation.isWindowedWrites(appliedPTransform), + equalTo(writeFiles.isWindowedWrites())); + + assertThat(WriteFilesTranslation.getSink(appliedPTransform), equalTo(writeFiles.getSink())); + } + } + + /** + * A simple {@link FileBasedSink} for testing serialization/deserialization. Not mocked to avoid + * any issues serializing mocks. + */ + private static class DummySink extends FileBasedSink { + + DummySink() { + super( + StaticValueProvider.of(FileSystems.matchNewResource("nowhere", false)), + new DummyFilenamePolicy()); + } + + @Override + public WriteOperation createWriteOperation() { + return new DummyWriteOperation(this); + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof DummySink)) { + return false; + } + + DummySink that = (DummySink) other; + + return getFilenamePolicy().equals(((DummySink) other).getFilenamePolicy()) + && getBaseOutputDirectoryProvider().isAccessible() + && that.getBaseOutputDirectoryProvider().isAccessible() + && getBaseOutputDirectoryProvider() + .get() + .equals(that.getBaseOutputDirectoryProvider().get()); + } + + @Override + public int hashCode() { + return Objects.hash( + DummySink.class, + getFilenamePolicy(), + getBaseOutputDirectoryProvider().isAccessible() + ? getBaseOutputDirectoryProvider().get() + : null); + } + } + + private static class DummyWriteOperation extends FileBasedSink.WriteOperation { + public DummyWriteOperation(FileBasedSink sink) { + super(sink); + } + + @Override + public FileBasedSink.Writer createWriter() throws Exception { + throw new UnsupportedOperationException("Should never be called."); + } + } + + private static class DummyFilenamePolicy extends FilenamePolicy { + @Override + public ResourceId windowedFilename( + ResourceId outputDirectory, WindowedContext c, String extension) { + throw new UnsupportedOperationException("Should never be called."); + } + + @Nullable + @Override + public ResourceId unwindowedFilename(ResourceId outputDirectory, Context c, String extension) { + throw new UnsupportedOperationException("Should never be called."); + } + + @Override + public boolean equals(Object other) { + return other instanceof DummyFilenamePolicy; + } + + @Override + public int hashCode() { + return DummyFilenamePolicy.class.hashCode(); + } + } +} From 87e0fdc43666a376d78af36e35e24d734da4c8a7 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Thu, 1 Jun 2017 11:12:56 -0700 Subject: [PATCH 138/159] fix python lint errors --- sdks/python/apache_beam/runners/portability/fn_api_runner.py | 2 +- sdks/python/apache_beam/runners/worker/data_plane_test.py | 2 +- sdks/python/apache_beam/runners/worker/log_handler_test.py | 2 +- sdks/python/apache_beam/runners/worker/sdk_worker_test.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner.py index 26355595e991..db34ef967176 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py @@ -23,8 +23,8 @@ import Queue as queue import threading -import grpc from concurrent import futures +import grpc import apache_beam as beam from apache_beam.coders import WindowedValueCoder diff --git a/sdks/python/apache_beam/runners/worker/data_plane_test.py b/sdks/python/apache_beam/runners/worker/data_plane_test.py index 7340789b88d9..e3e01ac5971f 100644 --- a/sdks/python/apache_beam/runners/worker/data_plane_test.py +++ b/sdks/python/apache_beam/runners/worker/data_plane_test.py @@ -26,8 +26,8 @@ import threading import unittest -import grpc from concurrent import futures +import grpc from apache_beam.runners.api import beam_fn_api_pb2 from apache_beam.runners.worker import data_plane diff --git a/sdks/python/apache_beam/runners/worker/log_handler_test.py b/sdks/python/apache_beam/runners/worker/log_handler_test.py index 565bedbd2385..8720ca8a3f8b 100644 --- a/sdks/python/apache_beam/runners/worker/log_handler_test.py +++ b/sdks/python/apache_beam/runners/worker/log_handler_test.py @@ -19,8 +19,8 @@ import logging import unittest -import grpc from concurrent import futures +import grpc from apache_beam.runners.api import beam_fn_api_pb2 from apache_beam.runners.worker import log_handler diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py index 996f44c7f4a0..0d0811b22201 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py @@ -24,8 +24,8 @@ import logging import unittest -import grpc from concurrent import futures +import grpc from apache_beam.io.concat_source_test import RangeSource from apache_beam.io.iobase import SourceBundle From 3dd2fb1e951bbfde8053bccef32aa73c51f9845d Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 25 May 2017 06:45:27 -0700 Subject: [PATCH 139/159] Add GroupByKey translation --- .../construction/GroupByKeyTranslation.java | 62 +++++++++++++++++++ .../GroupByKeyTranslationTest.java | 44 +++++++++++++ 2 files changed, 106 insertions(+) create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.java new file mode 100644 index 000000000000..db73461278d4 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/GroupByKeyTranslation.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.core.construction; + +import com.google.auto.service.AutoService; +import java.util.Collections; +import java.util.Map; +import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; +import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; + +/** + * Utility methods for translating a {@link GroupByKey} to and from {@link RunnerApi} + * representations. + */ +public class GroupByKeyTranslation { + + static class GroupByKeyTranslator implements TransformPayloadTranslator> { + @Override + public String getUrn(GroupByKey transform) { + return PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN; + } + + @Override + public FunctionSpec translate( + AppliedPTransform> transform, SdkComponents components) { + return FunctionSpec.newBuilder() + .setUrn(getUrn(transform.getTransform())) + .build(); + } + } + + + /** Registers {@link GroupByKeyTranslator}. */ + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class Registrar implements TransformPayloadTranslatorRegistrar { + @Override + public Map, ? extends TransformPayloadTranslator> + getTransformPayloadTranslators() { + return Collections.singletonMap(GroupByKey.class, new GroupByKeyTranslator()); + } + } +} diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java new file mode 100644 index 000000000000..22681f714bd3 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/GroupByKeyTranslationTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.core.construction; + +import static org.apache.beam.runners.core.construction.PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import org.apache.beam.sdk.transforms.GroupByKey; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link GroupByKeyTranslation}. */ +@RunWith(JUnit4.class) +public class GroupByKeyTranslationTest { + + /** + * Tests that the translator is registered so the URN can be retrieved (the only thing you can + * meaningfully do with a {@link GroupByKey}). + */ + @Test + public void testUrnRetrievable() throws Exception { + assertThat( + PTransformTranslation.urnForTransform(GroupByKey.create()), + equalTo(GROUP_BY_KEY_TRANSFORM_URN)); + } +} From 9b8052430324d010f3e6a56c085fd8b65e6250f4 Mon Sep 17 00:00:00 2001 From: jasonkuster Date: Tue, 30 May 2017 17:59:09 -0700 Subject: [PATCH 140/159] Fetch only relevant PR Per https://github.com/jenkinsci/ghprb-plugin/pull/521 we can be much more efficient about how we fetch PRs initially. --- .test-infra/jenkins/common_job_properties.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.test-infra/jenkins/common_job_properties.groovy b/.test-infra/jenkins/common_job_properties.groovy index 5ba12fe70e14..6d4d68b7a340 100644 --- a/.test-infra/jenkins/common_job_properties.groovy +++ b/.test-infra/jenkins/common_job_properties.groovy @@ -75,7 +75,7 @@ class common_job_properties { remote { url('https://github.com/apache/' + repositoryName + '.git') refspec('+refs/heads/*:refs/remotes/origin/* ' + - '+refs/pull/*:refs/remotes/origin/pr/*') + '+refs/pull/${ghprbPullId}/*:refs/remotes/origin/pr/${ghprbPullId}/*') } branch('${sha1}') extensions { From fa23fa2b0080461d0d860dc219283a9f90ebdb23 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Wed, 31 May 2017 10:10:06 -0700 Subject: [PATCH 141/159] [BEAM-1348] Add Runner API constructs to ProcessBundleDescriptor. This is towards removing the Fn API constructs once code has been migrated. --- sdks/common/fn-api/pom.xml | 5 ++++ .../fn-api/src/main/proto/beam_fn_api.proto | 25 ++++++++++++++++--- sdks/java/core/pom.xml | 7 ++++++ 3 files changed, 33 insertions(+), 4 deletions(-) diff --git a/sdks/common/fn-api/pom.xml b/sdks/common/fn-api/pom.xml index ded955911533..77a9ba52e40b 100644 --- a/sdks/common/fn-api/pom.xml +++ b/sdks/common/fn-api/pom.xml @@ -81,6 +81,11 @@ + + org.apache.beam + beam-sdks-common-runner-api + + com.google.protobuf protobuf-java diff --git a/sdks/common/fn-api/src/main/proto/beam_fn_api.proto b/sdks/common/fn-api/src/main/proto/beam_fn_api.proto index 79e1872f8eeb..9fe2b2fa21d8 100644 --- a/sdks/common/fn-api/src/main/proto/beam_fn_api.proto +++ b/sdks/common/fn-api/src/main/proto/beam_fn_api.proto @@ -37,6 +37,7 @@ package org.apache.beam.fn.v1; option java_package = "org.apache.beam.fn.v1"; option java_outer_classname = "BeamFnApi"; +import "beam_runner_api.proto"; import "google/protobuf/any.proto"; import "google/protobuf/timestamp.proto"; @@ -264,19 +265,35 @@ message RegisterRequest { message RegisterResponse { } -// A descriptor of references used when processing a bundle. -// Stable +// Definitions that should be used to construct the bundle processing graph. message ProcessBundleDescriptor { // (Required) A pipeline level unique id which can be used as a reference to // refer to this. string id = 1; - // (Required) A list of primitive transforms that should + // (Deprecated) A list of primitive transforms that should // be used to construct the bundle processing graph. repeated PrimitiveTransform primitive_transform = 2; - // (Required) The set of all coders referenced in this bundle. + // (Deprecated) The set of all coders referenced in this bundle. repeated Coder coders = 4; + + // (Required) A map from pipeline-scoped id to PTransform. + map transforms = 5; + + // (Required) A map from pipeline-scoped id to PCollection. + map pcollections = 6; + + // (Required) A map from pipeline-scoped id to WindowingStrategy. + map windowing_strategies = 7; + + // (Required) A map from pipeline-scoped id to Coder. + // TODO: Rename to "coders" once deprecated coders field is removed. Unique + // name is choosen to make it an easy search/replace + map codersyyy = 8; + + // (Required) A map from pipeline-scoped id to Environment. + map environments = 9; } // A request to process a given bundle. diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index 3c68c96e15fe..11b68e664eac 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -305,6 +305,13 @@ beam-sdks-common-fn-api test-jar test + + + + org.apache.beam + beam-sdks-common-runner-api + + From de757860945d5966a51173c54d29d0a733e66686 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 24 May 2017 17:23:31 -0700 Subject: [PATCH 142/159] Unify Java and Python WindowingStrategy representations. --- pom.xml | 6 ++ runners/core-construction-java/pom.xml | 5 ++ .../WindowingStrategyTranslation.java | 60 ++++++++++++++----- .../src/main/proto/beam_known_payloads.proto | 53 ++++++++++++++++ .../runners/dataflow/dataflow_runner.py | 39 +++++++++++- .../runners/dataflow/dataflow_runner_test.py | 11 ++++ sdks/python/apache_beam/transforms/window.py | 57 +++++++++++------- sdks/python/apache_beam/utils/proto_utils.py | 6 ++ sdks/python/apache_beam/utils/urns.py | 10 ++-- sdks/python/run_pylint.sh | 2 + 10 files changed, 206 insertions(+), 43 deletions(-) create mode 100644 sdks/common/runner-api/src/main/proto/beam_known_payloads.proto diff --git a/pom.xml b/pom.xml index 3e302e750864..805a8d64e9c8 100644 --- a/pom.xml +++ b/pom.xml @@ -944,6 +944,12 @@ ${protobuf.version} + + com.google.protobuf + protobuf-java-util + ${protobuf.version} + + com.google.api.grpc grpc-google-common-protos diff --git a/runners/core-construction-java/pom.xml b/runners/core-construction-java/pom.xml index 7eaa6f35b2f9..67951e9334f4 100644 --- a/runners/core-construction-java/pom.xml +++ b/runners/core-construction-java/pom.xml @@ -69,6 +69,11 @@ protobuf-java + + com.google.protobuf + protobuf-java-util + + com.google.code.findbugs jsr305 diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java index e92565f1a783..a226624ca4af 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java @@ -17,12 +17,12 @@ */ package org.apache.beam.runners.core.construction; -import static com.google.common.base.Preconditions.checkArgument; - import com.google.protobuf.Any; import com.google.protobuf.ByteString; import com.google.protobuf.BytesValue; import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.util.Durations; +import com.google.protobuf.util.Timestamps; import java.io.IOException; import java.io.Serializable; import org.apache.beam.sdk.common.runner.v1.RunnerApi; @@ -30,6 +30,11 @@ import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.common.runner.v1.RunnerApi.OutputTime; import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApiPayloads; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Sessions; +import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; @@ -153,9 +158,13 @@ public static TimestampCombiner timestampCombinerFromProto(RunnerApi.OutputTime } } - // This URN says that the WindowFn is just a UDF blob the indicated SDK understands + public static final String GLOBAL_WINDOWS_FN = "beam:windowfn:global_windows:v0.1"; + public static final String FIXED_WINDOWS_FN = "beam:windowfn:fixed_windows:v0.1"; + public static final String SLIDING_WINDOWS_FN = "beam:windowfn:sliding_windows:v0.1"; + public static final String SESSION_WINDOWS_FN = "beam:windowfn:session_windows:v0.1"; + // This URN says that the WindowFn is just a UDF blob the Java SDK understands // TODO: standardize such things - public static final String CUSTOM_WINDOWFN_URN = "urn:beam:windowfn:javasdk:0.1"; + public static final String SERIALIZED_JAVA_WINDOWFN_URN = "beam:windowfn:javasdk:v0.1"; /** * Converts a {@link WindowFn} into a {@link RunnerApi.MessageWithComponents} where {@link @@ -168,7 +177,7 @@ public static SdkFunctionSpec toProto( // TODO: Set environment ID .setSpec( FunctionSpec.newBuilder() - .setUrn(CUSTOM_WINDOWFN_URN) + .setUrn(SERIALIZED_JAVA_WINDOWFN_URN) .setParameter( Any.pack( BytesValue.newBuilder() @@ -261,18 +270,37 @@ public static RunnerApi.WindowingStrategy toProto( public static WindowFn windowFnFromProto(SdkFunctionSpec windowFnSpec) throws InvalidProtocolBufferException { - checkArgument( - windowFnSpec.getSpec().getUrn().equals(CUSTOM_WINDOWFN_URN), - "Only Java-serialized %s instances are supported, with URN %s. But found URN %s", - WindowFn.class.getSimpleName(), - CUSTOM_WINDOWFN_URN, - windowFnSpec.getSpec().getUrn()); - - Object deserializedWindowFn = - SerializableUtils.deserializeFromByteArray( + switch (windowFnSpec.getSpec().getUrn()) { + case GLOBAL_WINDOWS_FN: + return new GlobalWindows(); + case FIXED_WINDOWS_FN: + RunnerApiPayloads.FixedWindowsPayload fixedParams = + windowFnSpec.getSpec().getParameter().unpack( + RunnerApiPayloads.FixedWindowsPayload.class); + return FixedWindows.of( + Duration.millis(Durations.toMillis(fixedParams.getSize()))) + .withOffset(Duration.millis(Timestamps.toMillis(fixedParams.getOffset()))); + case SLIDING_WINDOWS_FN: + RunnerApiPayloads.SlidingWindowsPayload slidingParams = + windowFnSpec.getSpec().getParameter().unpack( + RunnerApiPayloads.SlidingWindowsPayload.class); + return SlidingWindows.of( + Duration.millis(Durations.toMillis(slidingParams.getSize()))) + .every(Duration.millis(Durations.toMillis(slidingParams.getPeriod()))) + .withOffset(Duration.millis(Timestamps.toMillis(slidingParams.getOffset()))); + case SESSION_WINDOWS_FN: + RunnerApiPayloads.SessionsPayload sessionParams = + windowFnSpec.getSpec().getParameter().unpack( + RunnerApiPayloads.SessionsPayload.class); + return Sessions.withGapDuration( + Duration.millis(Durations.toMillis(sessionParams.getGapSize()))); + case SERIALIZED_JAVA_WINDOWFN_URN: + return (WindowFn) SerializableUtils.deserializeFromByteArray( windowFnSpec.getSpec().getParameter().unpack(BytesValue.class).getValue().toByteArray(), "WindowFn"); - - return (WindowFn) deserializedWindowFn; + default: + throw new IllegalArgumentException( + "Unknown or unsupported WindowFn: " + windowFnSpec.getSpec().getUrn()); + } } } diff --git a/sdks/common/runner-api/src/main/proto/beam_known_payloads.proto b/sdks/common/runner-api/src/main/proto/beam_known_payloads.proto new file mode 100644 index 000000000000..446bd594bd06 --- /dev/null +++ b/sdks/common/runner-api/src/main/proto/beam_known_payloads.proto @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Protocol Buffers describing the Runner API, which is the runner-independent, + * SDK-independent definition of the Beam model. + */ + +syntax = "proto3"; + +package org.apache.beam.runner_api.v1; + +option java_package = "org.apache.beam.sdk.common.runner.v1"; +option java_outer_classname = "RunnerApiPayloads"; + +import "google/protobuf/duration.proto"; +import "google/protobuf/timestamp.proto"; + +// beam:windowfn:global_windows:v0.1 +// empty payload + +// beam:windowfn:fixed_windows:v0.1 +message FixedWindowsPayload { + google.protobuf.Duration size = 1; + google.protobuf.Timestamp offset = 2; +} + +// beam:windowfn:sliding_windows:v0.1 +message SlidingWindowsPayload { + google.protobuf.Duration size = 1; + google.protobuf.Timestamp offset = 2; + google.protobuf.Duration period = 3; +} + +// beam:windowfn:session_windows:v0.1 +message SessionsPayload { + google.protobuf.Duration gap_size = 1; +} diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 046d3d585388..3e0e26865d2d 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -25,6 +25,7 @@ import threading import time import traceback +import urllib from apache_beam import error from apache_beam import coders @@ -416,7 +417,9 @@ def run_GroupByKey(self, transform_node): PropertyNames.OUTPUT_NAME: PropertyNames.OUT}]) windowing = transform_node.transform.get_windowing( transform_node.inputs) - step.add_property(PropertyNames.SERIALIZED_FN, pickler.dumps(windowing)) + step.add_property( + PropertyNames.SERIALIZED_FN, + self.serialize_windowing_strategy(windowing)) def run_ParDo(self, transform_node): transform = transform_node.transform @@ -697,6 +700,40 @@ def run__NativeWrite(self, transform_node): PropertyNames.STEP_NAME: input_step.proto.name, PropertyNames.OUTPUT_NAME: input_step.get_output(input_tag)}) + @classmethod + def serialize_windowing_strategy(cls, windowing): + from apache_beam.runners import pipeline_context + from apache_beam.runners.api import beam_runner_api_pb2 + context = pipeline_context.PipelineContext() + windowing_proto = windowing.to_runner_api(context) + return cls.byte_array_to_json_string( + beam_runner_api_pb2.MessageWithComponents( + components=context.to_runner_api(), + windowing_strategy=windowing_proto).SerializeToString()) + + @classmethod + def deserialize_windowing_strategy(cls, serialized_data): + # Imported here to avoid circular dependencies. + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam.runners import pipeline_context + from apache_beam.runners.api import beam_runner_api_pb2 + from apache_beam.transforms.core import Windowing + proto = beam_runner_api_pb2.MessageWithComponents() + proto.ParseFromString(cls.json_string_to_byte_array(serialized_data)) + return Windowing.from_runner_api( + proto.windowing_strategy, + pipeline_context.PipelineContext(proto.components)) + + @staticmethod + def byte_array_to_json_string(raw_bytes): + """Implements org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString.""" + return urllib.quote(raw_bytes) + + @staticmethod + def json_string_to_byte_array(encoded_string): + """Implements org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray.""" + return urllib.unquote(encoded_string) + class DataflowPipelineResult(PipelineResult): """Represents the state of a pipeline run on the Dataflow service.""" diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index ff4b51d5a826..74fd01df7bc2 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -38,6 +38,8 @@ from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.transforms.display import DisplayDataItem from apache_beam.transforms.core import _GroupByKeyOnly +from apache_beam.transforms.core import Windowing +from apache_beam.transforms import window from apache_beam.typehints import typehints # Protect against environments where apitools library is not available. @@ -240,6 +242,15 @@ def _test_flatten_input_visitor(self, input_type, output_type, num_inputs): for _ in range(num_inputs): self.assertEqual(inputs[0].element_type, output_type) + def test_serialize_windowing_strategy(self): + # This just tests the basic path; more complete tests + # are in window_test.py. + strategy = Windowing(window.FixedWindows(10)) + self.assertEqual( + strategy, + DataflowRunner.deserialize_windowing_strategy( + DataflowRunner.serialize_windowing_strategy(strategy))) + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/transforms/window.py b/sdks/python/apache_beam/transforms/window.py index 94187e0c5d54..f74c8a931ddf 100644 --- a/sdks/python/apache_beam/transforms/window.py +++ b/sdks/python/apache_beam/transforms/window.py @@ -51,10 +51,12 @@ import abc -from google.protobuf import struct_pb2 +from google.protobuf import duration_pb2 +from google.protobuf import timestamp_pb2 from apache_beam.coders import coders from apache_beam.runners.api import beam_runner_api_pb2 +from apache_beam.runners.api import beam_known_payloads_pb2 from apache_beam.transforms import timeutil from apache_beam.utils import proto_utils from apache_beam.utils import urns @@ -341,14 +343,18 @@ def __ne__(self, other): def to_runner_api_parameter(self, context): return (urns.FIXED_WINDOWS_FN, - proto_utils.pack_Struct(size=self.size.micros, - offset=self.offset.micros)) - - @urns.RunnerApiFn.register_urn(urns.FIXED_WINDOWS_FN, struct_pb2.Struct) + beam_known_payloads_pb2.FixedWindowsPayload( + size=proto_utils.from_micros( + duration_pb2.Duration, self.size.micros), + offset=proto_utils.from_micros( + timestamp_pb2.Timestamp, self.offset.micros))) + + @urns.RunnerApiFn.register_urn( + urns.FIXED_WINDOWS_FN, beam_known_payloads_pb2.FixedWindowsPayload) def from_runner_api_parameter(fn_parameter, unused_context): return FixedWindows( - size=Duration(micros=fn_parameter['size']), - offset=Timestamp(micros=fn_parameter['offset'])) + size=Duration(micros=fn_parameter.size.ToMicroseconds()), + offset=Timestamp(micros=fn_parameter.offset.ToMicroseconds())) class SlidingWindows(NonMergingWindowFn): @@ -392,17 +398,22 @@ def __eq__(self, other): def to_runner_api_parameter(self, context): return (urns.SLIDING_WINDOWS_FN, - proto_utils.pack_Struct( - size=self.size.micros, - offset=self.offset.micros, - period=self.period.micros)) - - @urns.RunnerApiFn.register_urn(urns.SLIDING_WINDOWS_FN, struct_pb2.Struct) + beam_known_payloads_pb2.SlidingWindowsPayload( + size=proto_utils.from_micros( + duration_pb2.Duration, self.size.micros), + offset=proto_utils.from_micros( + timestamp_pb2.Timestamp, self.offset.micros), + period=proto_utils.from_micros( + duration_pb2.Duration, self.period.micros))) + + @urns.RunnerApiFn.register_urn( + urns.SLIDING_WINDOWS_FN, + beam_known_payloads_pb2.SlidingWindowsPayload) def from_runner_api_parameter(fn_parameter, unused_context): return SlidingWindows( - size=Duration(micros=fn_parameter['size']), - offset=Timestamp(micros=fn_parameter['offset']), - period=Duration(micros=fn_parameter['period'])) + size=Duration(micros=fn_parameter.size.ToMicroseconds()), + offset=Timestamp(micros=fn_parameter.offset.ToMicroseconds()), + period=Duration(micros=fn_parameter.period.ToMicroseconds())) class Sessions(WindowFn): @@ -452,10 +463,14 @@ def __eq__(self, other): if type(self) == type(other) == Sessions: return self.gap_size == other.gap_size - @urns.RunnerApiFn.register_urn(urns.SESSION_WINDOWS_FN, struct_pb2.Struct) - def from_runner_api_parameter(fn_parameter, unused_context): - return Sessions(gap_size=Duration(micros=fn_parameter['gap_size'])) - def to_runner_api_parameter(self, context): return (urns.SESSION_WINDOWS_FN, - proto_utils.pack_Struct(gap_size=self.gap_size.micros)) + beam_known_payloads_pb2.SessionsPayload( + gap_size=proto_utils.from_micros( + duration_pb2.Duration, self.gap_size.micros))) + + @urns.RunnerApiFn.register_urn( + urns.SESSION_WINDOWS_FN, beam_known_payloads_pb2.SessionsPayload) + def from_runner_api_parameter(fn_parameter, unused_context): + return Sessions( + gap_size=Duration(micros=fn_parameter.gap_size.ToMicroseconds())) diff --git a/sdks/python/apache_beam/utils/proto_utils.py b/sdks/python/apache_beam/utils/proto_utils.py index 090a8212f4fc..af8f21897b87 100644 --- a/sdks/python/apache_beam/utils/proto_utils.py +++ b/sdks/python/apache_beam/utils/proto_utils.py @@ -53,3 +53,9 @@ def pack_Struct(**kwargs): for key, value in kwargs.items(): msg[key] = value # pylint: disable=unsubscriptable-object, unsupported-assignment-operation return msg + + +def from_micros(cls, micros): + result = cls() + result.FromMicroseconds(micros) + return result diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py index 379b5ff9f20b..849b8e37dfbd 100644 --- a/sdks/python/apache_beam/utils/urns.py +++ b/sdks/python/apache_beam/utils/urns.py @@ -26,11 +26,11 @@ from apache_beam.utils import proto_utils -PICKLED_WINDOW_FN = "beam:window_fn:pickled_python:v0.1" -GLOBAL_WINDOWS_FN = "beam:window_fn:global_windows:v0.1" -FIXED_WINDOWS_FN = "beam:window_fn:fixed_windows:v0.1" -SLIDING_WINDOWS_FN = "beam:window_fn:sliding_windows:v0.1" -SESSION_WINDOWS_FN = "beam:window_fn:session_windows:v0.1" +PICKLED_WINDOW_FN = "beam:windowfn:pickled_python:v0.1" +GLOBAL_WINDOWS_FN = "beam:windowfn:global_windows:v0.1" +FIXED_WINDOWS_FN = "beam:windowfn:fixed_windows:v0.1" +SLIDING_WINDOWS_FN = "beam:windowfn:sliding_windows:v0.1" +SESSION_WINDOWS_FN = "beam:windowfn:session_windows:v0.1" PICKLED_CODER = "beam:coder:pickled_python:v0.1" diff --git a/sdks/python/run_pylint.sh b/sdks/python/run_pylint.sh index 400c577519dd..4ef3e7fb438d 100755 --- a/sdks/python/run_pylint.sh +++ b/sdks/python/run_pylint.sh @@ -50,6 +50,8 @@ EXCLUDED_GENERATED_FILES=( "apache_beam/runners/api/beam_fn_api_pb2_grpc.py" "apache_beam/runners/api/beam_runner_api_pb2.py" "apache_beam/runners/api/beam_runner_api_pb2_grpc.py" +"apache_beam/runners/api/beam_known_payloads_pb2.py" +"apache_beam/runners/api/beam_known_payloads_pb2_grpc.py" ) FILES_TO_IGNORE="" From b490e84ef0b4e56cabc091cfe2cc42f8f1e69caa Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 25 May 2017 13:26:02 -0700 Subject: [PATCH 143/159] Add known window serialization for Java. --- .../WindowingStrategyTranslation.java | 105 ++++++++++++++---- .../WindowingStrategyTranslationTest.java | 9 ++ ...yloads.proto => standard_window_fns.proto} | 2 +- sdks/python/apache_beam/transforms/window.py | 14 +-- 4 files changed, 102 insertions(+), 28 deletions(-) rename sdks/common/runner-api/src/main/proto/{beam_known_payloads.proto => standard_window_fns.proto} (96%) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java index a226624ca4af..718efe7c9190 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java @@ -30,7 +30,7 @@ import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.common.runner.v1.RunnerApi.OutputTime; import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; -import org.apache.beam.sdk.common.runner.v1.RunnerApiPayloads; +import org.apache.beam.sdk.common.runner.v1.StandardWindowFns; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Sessions; @@ -165,6 +165,9 @@ public static TimestampCombiner timestampCombinerFromProto(RunnerApi.OutputTime // This URN says that the WindowFn is just a UDF blob the Java SDK understands // TODO: standardize such things public static final String SERIALIZED_JAVA_WINDOWFN_URN = "beam:windowfn:javasdk:v0.1"; + public static final String OLD_SERIALIZED_JAVA_WINDOWFN_URN = "urn:beam:windowfn:javasdk:0.1"; + // Remove this once the dataflow worker understands all the above formats. + private static final boolean USE_OLD_SERIALIZED_JAVA_WINDOWFN_URN = true; /** * Converts a {@link WindowFn} into a {@link RunnerApi.MessageWithComponents} where {@link @@ -173,19 +176,80 @@ public static TimestampCombiner timestampCombinerFromProto(RunnerApi.OutputTime */ public static SdkFunctionSpec toProto( WindowFn windowFn, @SuppressWarnings("unused") SdkComponents components) { - return SdkFunctionSpec.newBuilder() - // TODO: Set environment ID - .setSpec( - FunctionSpec.newBuilder() - .setUrn(SERIALIZED_JAVA_WINDOWFN_URN) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom( - SerializableUtils.serializeToByteArray(windowFn))) - .build()))) - .build(); + // TODO: Set environment IDs + if (USE_OLD_SERIALIZED_JAVA_WINDOWFN_URN) { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(OLD_SERIALIZED_JAVA_WINDOWFN_URN) + .setParameter( + Any.pack( + BytesValue.newBuilder() + .setValue( + ByteString.copyFrom( + SerializableUtils.serializeToByteArray(windowFn))) + .build()))) + .build(); + } else if (windowFn instanceof GlobalWindows) { + return SdkFunctionSpec.newBuilder() + .setSpec(FunctionSpec.newBuilder().setUrn(GLOBAL_WINDOWS_FN)) + .build(); + } else if (windowFn instanceof FixedWindows) { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(FIXED_WINDOWS_FN) + .setParameter( + Any.pack( + StandardWindowFns.FixedWindowsPayload.newBuilder() + .setSize(Durations.fromMillis( + ((FixedWindows) windowFn).getSize().getMillis())) + .setOffset(Timestamps.fromMillis( + ((FixedWindows) windowFn).getOffset().getMillis())) + .build()))) + .build(); + } else if (windowFn instanceof SlidingWindows) { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(SLIDING_WINDOWS_FN) + .setParameter( + Any.pack( + StandardWindowFns.SlidingWindowsPayload.newBuilder() + .setSize(Durations.fromMillis( + ((SlidingWindows) windowFn).getSize().getMillis())) + .setOffset(Timestamps.fromMillis( + ((SlidingWindows) windowFn).getOffset().getMillis())) + .setPeriod(Durations.fromMillis( + ((SlidingWindows) windowFn).getPeriod().getMillis())) + .build()))) + .build(); + } else if (windowFn instanceof Sessions) { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(SESSION_WINDOWS_FN) + .setParameter( + Any.pack( + StandardWindowFns.SessionsPayload.newBuilder() + .setGapSize(Durations.fromMillis( + ((Sessions) windowFn).getGapDuration().getMillis())) + .build()))) + .build(); + } else { + return SdkFunctionSpec.newBuilder() + .setSpec( + FunctionSpec.newBuilder() + .setUrn(SERIALIZED_JAVA_WINDOWFN_URN) + .setParameter( + Any.pack( + BytesValue.newBuilder() + .setValue( + ByteString.copyFrom( + SerializableUtils.serializeToByteArray(windowFn))) + .build()))) + .build(); + } } /** @@ -274,27 +338,28 @@ public static RunnerApi.WindowingStrategy toProto( case GLOBAL_WINDOWS_FN: return new GlobalWindows(); case FIXED_WINDOWS_FN: - RunnerApiPayloads.FixedWindowsPayload fixedParams = + StandardWindowFns.FixedWindowsPayload fixedParams = windowFnSpec.getSpec().getParameter().unpack( - RunnerApiPayloads.FixedWindowsPayload.class); + StandardWindowFns.FixedWindowsPayload.class); return FixedWindows.of( Duration.millis(Durations.toMillis(fixedParams.getSize()))) .withOffset(Duration.millis(Timestamps.toMillis(fixedParams.getOffset()))); case SLIDING_WINDOWS_FN: - RunnerApiPayloads.SlidingWindowsPayload slidingParams = + StandardWindowFns.SlidingWindowsPayload slidingParams = windowFnSpec.getSpec().getParameter().unpack( - RunnerApiPayloads.SlidingWindowsPayload.class); + StandardWindowFns.SlidingWindowsPayload.class); return SlidingWindows.of( Duration.millis(Durations.toMillis(slidingParams.getSize()))) .every(Duration.millis(Durations.toMillis(slidingParams.getPeriod()))) .withOffset(Duration.millis(Timestamps.toMillis(slidingParams.getOffset()))); case SESSION_WINDOWS_FN: - RunnerApiPayloads.SessionsPayload sessionParams = + StandardWindowFns.SessionsPayload sessionParams = windowFnSpec.getSpec().getParameter().unpack( - RunnerApiPayloads.SessionsPayload.class); + StandardWindowFns.SessionsPayload.class); return Sessions.withGapDuration( Duration.millis(Durations.toMillis(sessionParams.getGapSize()))); case SERIALIZED_JAVA_WINDOWFN_URN: + case OLD_SERIALIZED_JAVA_WINDOWFN_URN: return (WindowFn) SerializableUtils.deserializeFromByteArray( windowFnSpec.getSpec().getParameter().unpack(BytesValue.class).getValue().toByteArray(), "WindowFn"); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java index 1e528039562c..e40654546787 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java @@ -25,6 +25,8 @@ import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Sessions; +import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; @@ -62,6 +64,13 @@ private static ToProtoAndBackSpec toProtoAndBackSpec(WindowingStrategy windowing public static Iterable data() { return ImmutableList.of( toProtoAndBackSpec(WindowingStrategy.globalDefault()), + toProtoAndBackSpec(WindowingStrategy.of( + FixedWindows.of(Duration.millis(11)).withOffset(Duration.millis(3)))), + toProtoAndBackSpec(WindowingStrategy.of( + SlidingWindows.of(Duration.millis(37)).every(Duration.millis(3)) + .withOffset(Duration.millis(2)))), + toProtoAndBackSpec(WindowingStrategy.of( + Sessions.withGapDuration(Duration.millis(389)))), toProtoAndBackSpec( WindowingStrategy.of(REPRESENTATIVE_WINDOW_FN) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS) diff --git a/sdks/common/runner-api/src/main/proto/beam_known_payloads.proto b/sdks/common/runner-api/src/main/proto/standard_window_fns.proto similarity index 96% rename from sdks/common/runner-api/src/main/proto/beam_known_payloads.proto rename to sdks/common/runner-api/src/main/proto/standard_window_fns.proto index 446bd594bd06..0682044e3791 100644 --- a/sdks/common/runner-api/src/main/proto/beam_known_payloads.proto +++ b/sdks/common/runner-api/src/main/proto/standard_window_fns.proto @@ -26,7 +26,7 @@ syntax = "proto3"; package org.apache.beam.runner_api.v1; option java_package = "org.apache.beam.sdk.common.runner.v1"; -option java_outer_classname = "RunnerApiPayloads"; +option java_outer_classname = "StandardWindowFns"; import "google/protobuf/duration.proto"; import "google/protobuf/timestamp.proto"; diff --git a/sdks/python/apache_beam/transforms/window.py b/sdks/python/apache_beam/transforms/window.py index f74c8a931ddf..e87a00763897 100644 --- a/sdks/python/apache_beam/transforms/window.py +++ b/sdks/python/apache_beam/transforms/window.py @@ -56,7 +56,7 @@ from apache_beam.coders import coders from apache_beam.runners.api import beam_runner_api_pb2 -from apache_beam.runners.api import beam_known_payloads_pb2 +from apache_beam.runners.api import standard_window_fns_pb2 from apache_beam.transforms import timeutil from apache_beam.utils import proto_utils from apache_beam.utils import urns @@ -343,14 +343,14 @@ def __ne__(self, other): def to_runner_api_parameter(self, context): return (urns.FIXED_WINDOWS_FN, - beam_known_payloads_pb2.FixedWindowsPayload( + standard_window_fns_pb2.FixedWindowsPayload( size=proto_utils.from_micros( duration_pb2.Duration, self.size.micros), offset=proto_utils.from_micros( timestamp_pb2.Timestamp, self.offset.micros))) @urns.RunnerApiFn.register_urn( - urns.FIXED_WINDOWS_FN, beam_known_payloads_pb2.FixedWindowsPayload) + urns.FIXED_WINDOWS_FN, standard_window_fns_pb2.FixedWindowsPayload) def from_runner_api_parameter(fn_parameter, unused_context): return FixedWindows( size=Duration(micros=fn_parameter.size.ToMicroseconds()), @@ -398,7 +398,7 @@ def __eq__(self, other): def to_runner_api_parameter(self, context): return (urns.SLIDING_WINDOWS_FN, - beam_known_payloads_pb2.SlidingWindowsPayload( + standard_window_fns_pb2.SlidingWindowsPayload( size=proto_utils.from_micros( duration_pb2.Duration, self.size.micros), offset=proto_utils.from_micros( @@ -408,7 +408,7 @@ def to_runner_api_parameter(self, context): @urns.RunnerApiFn.register_urn( urns.SLIDING_WINDOWS_FN, - beam_known_payloads_pb2.SlidingWindowsPayload) + standard_window_fns_pb2.SlidingWindowsPayload) def from_runner_api_parameter(fn_parameter, unused_context): return SlidingWindows( size=Duration(micros=fn_parameter.size.ToMicroseconds()), @@ -465,12 +465,12 @@ def __eq__(self, other): def to_runner_api_parameter(self, context): return (urns.SESSION_WINDOWS_FN, - beam_known_payloads_pb2.SessionsPayload( + standard_window_fns_pb2.SessionsPayload( gap_size=proto_utils.from_micros( duration_pb2.Duration, self.gap_size.micros))) @urns.RunnerApiFn.register_urn( - urns.SESSION_WINDOWS_FN, beam_known_payloads_pb2.SessionsPayload) + urns.SESSION_WINDOWS_FN, standard_window_fns_pb2.SessionsPayload) def from_runner_api_parameter(fn_parameter, unused_context): return Sessions( gap_size=Duration(micros=fn_parameter.gap_size.ToMicroseconds())) From 4c20bfa9929664031f3fce67a723d09fc629ad3a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 31 May 2017 15:53:47 -0700 Subject: [PATCH 144/159] Ignore all proto generated files. --- sdks/python/run_pylint.sh | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/sdks/python/run_pylint.sh b/sdks/python/run_pylint.sh index 4ef3e7fb438d..7808136a4925 100755 --- a/sdks/python/run_pylint.sh +++ b/sdks/python/run_pylint.sh @@ -46,12 +46,7 @@ EXCLUDED_GENERATED_FILES=( "apache_beam/io/gcp/internal/clients/storage/storage_v1_client.py" "apache_beam/io/gcp/internal/clients/storage/storage_v1_messages.py" "apache_beam/coders/proto2_coder_test_messages_pb2.py" -"apache_beam/runners/api/beam_fn_api_pb2.py" -"apache_beam/runners/api/beam_fn_api_pb2_grpc.py" -"apache_beam/runners/api/beam_runner_api_pb2.py" -"apache_beam/runners/api/beam_runner_api_pb2_grpc.py" -"apache_beam/runners/api/beam_known_payloads_pb2.py" -"apache_beam/runners/api/beam_known_payloads_pb2_grpc.py" +apache_beam/runners/api/*pb2*.py ) FILES_TO_IGNORE="" From 99907b94c074d4dc2ac3911b690da6790056412a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 1 Jun 2017 16:01:56 -0700 Subject: [PATCH 145/159] Increase dataflow worker version. --- sdks/python/apache_beam/runners/dataflow/internal/dependency.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/dependency.py b/sdks/python/apache_beam/runners/dataflow/internal/dependency.py index e69c8d7dac50..3a0ff46bfe52 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/dependency.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/dependency.py @@ -73,7 +73,7 @@ # Update this version to the next version whenever there is a change that will # require changes to the execution environment. # This should be in the beam-[version]-[date] format, date is optional. -BEAM_CONTAINER_VERSION = 'beam-2.1.0-20170518' +BEAM_CONTAINER_VERSION = 'beam-2.1.0-20170601' # Standard file names used for staging files. WORKFLOW_TARBALL_FILE = 'workflow.tar.gz' From c5918b2f7ce36c755e2a285c42cc6b628b9ee319 Mon Sep 17 00:00:00 2001 From: James Xu Date: Wed, 31 May 2017 10:28:55 +0800 Subject: [PATCH 146/159] [BEAM-2378] support FULL OUTER JOIN --- .../beam/sdk/extensions/joinlibrary/Join.java | 65 ++++++- .../joinlibrary/OuterFullJoinTest.java | 179 ++++++++++++++++++ 2 files changed, 243 insertions(+), 1 deletion(-) create mode 100644 sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterFullJoinTest.java diff --git a/sdks/java/extensions/join-library/src/main/java/org/apache/beam/sdk/extensions/joinlibrary/Join.java b/sdks/java/extensions/join-library/src/main/java/org/apache/beam/sdk/extensions/joinlibrary/Join.java index f4e6ccbe6dfe..9acb048f0cf7 100644 --- a/sdks/java/extensions/join-library/src/main/java/org/apache/beam/sdk/extensions/joinlibrary/Join.java +++ b/sdks/java/extensions/join-library/src/main/java/org/apache/beam/sdk/extensions/joinlibrary/Join.java @@ -141,7 +141,7 @@ public void processElement(ProcessContext c) { * @param Type of the values for the left collection. * @param Type of the values for the right collection. * @return A joined collection of KV where Key is the key and value is a - * KV where Key is of type V1 and Value is type V2. Keys that + * KV where Key is of type V1 and Value is type V2. Values that * should be null or empty is replaced with nullValue. */ public static PCollection>> rightOuterJoin( @@ -184,4 +184,67 @@ public void processElement(ProcessContext c) { KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), ((KvCoder) rightCollection.getCoder()).getValueCoder()))); } + + /** + * Full Outer Join of two collections of KV elements. + * @param leftCollection Left side collection to join. + * @param rightCollection Right side collection to join. + * @param leftNullValue Value to use as null value when left side do not match right side. + * @param rightNullValue Value to use as null value when right side do not match right side. + * @param Type of the key for both collections + * @param Type of the values for the left collection. + * @param Type of the values for the right collection. + * @return A joined collection of KV where Key is the key and value is a + * KV where Key is of type V1 and Value is type V2. Values that + * should be null or empty is replaced with leftNullValue/rightNullValue. + */ + public static PCollection>> fullOuterJoin( + final PCollection> leftCollection, + final PCollection> rightCollection, + final V1 leftNullValue, final V2 rightNullValue) { + checkNotNull(leftCollection); + checkNotNull(rightCollection); + checkNotNull(leftNullValue); + checkNotNull(rightNullValue); + + final TupleTag v1Tuple = new TupleTag<>(); + final TupleTag v2Tuple = new TupleTag<>(); + + PCollection> coGbkResultCollection = + KeyedPCollectionTuple.of(v1Tuple, leftCollection) + .and(v2Tuple, rightCollection) + .apply(CoGroupByKey.create()); + + return coGbkResultCollection.apply(ParDo.of( + new DoFn, KV>>() { + @ProcessElement + public void processElement(ProcessContext c) { + KV e = c.element(); + + Iterable leftValuesIterable = e.getValue().getAll(v1Tuple); + Iterable rightValuesIterable = e.getValue().getAll(v2Tuple); + if (leftValuesIterable.iterator().hasNext() + && rightValuesIterable.iterator().hasNext()) { + for (V2 rightValue : rightValuesIterable) { + for (V1 leftValue : leftValuesIterable) { + c.output(KV.of(e.getKey(), KV.of(leftValue, rightValue))); + } + } + } else if (leftValuesIterable.iterator().hasNext() + && !rightValuesIterable.iterator().hasNext()) { + for (V1 leftValue : leftValuesIterable) { + c.output(KV.of(e.getKey(), KV.of(leftValue, rightNullValue))); + } + } else if (!leftValuesIterable.iterator().hasNext() + && rightValuesIterable.iterator().hasNext()) { + for (V2 rightValue : rightValuesIterable) { + c.output(KV.of(e.getKey(), KV.of(leftNullValue, rightValue))); + } + } + } + })) + .setCoder(KvCoder.of(((KvCoder) leftCollection.getCoder()).getKeyCoder(), + KvCoder.of(((KvCoder) leftCollection.getCoder()).getValueCoder(), + ((KvCoder) rightCollection.getCoder()).getValueCoder()))); + } } diff --git a/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterFullJoinTest.java b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterFullJoinTest.java new file mode 100644 index 000000000000..cdf4f4f77936 --- /dev/null +++ b/sdks/java/extensions/join-library/src/test/java/org/apache/beam/sdk/extensions/joinlibrary/OuterFullJoinTest.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.extensions.joinlibrary; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; +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.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +/** + * This test Outer Full Join functionality. + */ +public class OuterFullJoinTest { + + List> leftListOfKv; + List> listRightOfKv; + List>> expectedResult; + + @Rule + public final transient TestPipeline p = TestPipeline.create(); + + @Before + public void setup() { + + leftListOfKv = new ArrayList<>(); + listRightOfKv = new ArrayList<>(); + + expectedResult = new ArrayList<>(); + } + + @Test + public void testJoinOneToOneMapping() { + leftListOfKv.add(KV.of("Key1", 5L)); + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p + .apply("CreateLeft", Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key1", "foo")); + listRightOfKv.add(KV.of("Key2", "bar")); + PCollection> rightCollection = p + .apply("CreateRight", Create.of(listRightOfKv)); + + PCollection>> output = Join.fullOuterJoin( + leftCollection, rightCollection, -1L, ""); + + expectedResult.add(KV.of("Key1", KV.of(5L, "foo"))); + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); + PAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinOneToManyMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p + .apply("CreateLeft", Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key2", "bar")); + listRightOfKv.add(KV.of("Key2", "gazonk")); + PCollection> rightCollection = p + .apply("CreateRight", Create.of(listRightOfKv)); + + PCollection>> output = Join.fullOuterJoin( + leftCollection, rightCollection, -1L, ""); + + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); + expectedResult.add(KV.of("Key2", KV.of(4L, "gazonk"))); + PAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinManyToOneMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + leftListOfKv.add(KV.of("Key2", 6L)); + PCollection> leftCollection = p + .apply("CreateLeft", Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key2", "bar")); + PCollection> rightCollection = p + .apply("CreateRight", Create.of(listRightOfKv)); + + PCollection>> output = Join.fullOuterJoin( + leftCollection, rightCollection, -1L, ""); + + expectedResult.add(KV.of("Key2", KV.of(4L, "bar"))); + expectedResult.add(KV.of("Key2", KV.of(6L, "bar"))); + PAssert.that(output).containsInAnyOrder(expectedResult); + + p.run(); + } + + @Test + public void testJoinNoneToNoneMapping() { + leftListOfKv.add(KV.of("Key2", 4L)); + PCollection> leftCollection = p + .apply("CreateLeft", Create.of(leftListOfKv)); + + listRightOfKv.add(KV.of("Key3", "bar")); + PCollection> rightCollection = p + .apply("CreateRight", Create.of(listRightOfKv)); + + PCollection>> output = Join.fullOuterJoin( + leftCollection, rightCollection, -1L, ""); + + expectedResult.add(KV.of("Key2", KV.of(4L, ""))); + expectedResult.add(KV.of("Key3", KV.of(-1L, "bar"))); + PAssert.that(output).containsInAnyOrder(expectedResult); + p.run(); + } + + @Test(expected = NullPointerException.class) + public void testJoinLeftCollectionNull() { + p.enableAbandonedNodeEnforcement(false); + Join.fullOuterJoin( + null, + p.apply( + Create.of(listRightOfKv) + .withCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()))), + "", ""); + } + + @Test(expected = NullPointerException.class) + public void testJoinRightCollectionNull() { + p.enableAbandonedNodeEnforcement(false); + Join.fullOuterJoin( + p.apply( + Create.of(leftListOfKv).withCoder(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))), + null, + -1L, -1L); + } + + @Test(expected = NullPointerException.class) + public void testJoinLeftNullValueIsNull() { + p.enableAbandonedNodeEnforcement(false); + Join.fullOuterJoin( + p.apply("CreateLeft", Create.empty(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))), + p.apply( + "CreateRight", Create.empty(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()))), + null, ""); + } + + @Test(expected = NullPointerException.class) + public void testJoinRightNullValueIsNull() { + p.enableAbandonedNodeEnforcement(false); + Join.fullOuterJoin( + p.apply("CreateLeft", Create.empty(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))), + p.apply( + "CreateRight", Create.empty(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()))), + -1L, null); + } +} From c8fce2faf4ca380530f563adf4992c8bf3a5f3a5 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 30 May 2017 18:04:36 -0700 Subject: [PATCH 147/159] Bumps dataflow container version to 0530 --- runners/google-cloud-dataflow-java/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 16e12664918c..92c94a8394db 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -33,7 +33,7 @@ jar - beam-master-20170519 + beam-master-20170530 1 6 From e201c37cbbb2c1f979d3e202b8e0d2d6564f3d2d Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 26 May 2017 16:11:20 -0700 Subject: [PATCH 148/159] Removes unused PropertyNames --- .../runners/dataflow/util/PropertyNames.java | 46 ------------------- 1 file changed, 46 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PropertyNames.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PropertyNames.java index c8c990302102..f82f1f112c17 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PropertyNames.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PropertyNames.java @@ -21,67 +21,31 @@ * Constant property names used by the SDK in CloudWorkflow specifications. */ public class PropertyNames { - public static final String ALLOWED_ENCODINGS = "allowed_encodings"; - public static final String APPEND_TRAILING_NEWLINES = "append_trailing_newlines"; - public static final String BIGQUERY_CREATE_DISPOSITION = "create_disposition"; - public static final String BIGQUERY_DATASET = "dataset"; - public static final String BIGQUERY_PROJECT = "project"; - public static final String BIGQUERY_SCHEMA = "schema"; - public static final String BIGQUERY_TABLE = "table"; - public static final String BIGQUERY_QUERY = "bigquery_query"; - public static final String BIGQUERY_FLATTEN_RESULTS = "bigquery_flatten_results"; - public static final String BIGQUERY_USE_LEGACY_SQL = "bigquery_use_legacy_sql"; - public static final String BIGQUERY_WRITE_DISPOSITION = "write_disposition"; - public static final String BIGQUERY_EXPORT_FORMAT = "bigquery_export_format"; - public static final String BIGQUERY_EXPORT_SCHEMA = "bigquery_export_schema"; public static final String CO_GBK_RESULT_SCHEMA = "co_gbk_result_schema"; - public static final String COMBINE_FN = "combine_fn"; public static final String COMPONENT_ENCODINGS = "component_encodings"; - public static final String COMPRESSION_TYPE = "compression_type"; public static final String CUSTOM_SOURCE_FORMAT = "custom_source"; - public static final String CONCAT_SOURCE_SOURCES = "sources"; - public static final String CONCAT_SOURCE_BASE_SPECS = "base_specs"; public static final String SOURCE_STEP_INPUT = "custom_source_step_input"; public static final String SOURCE_SPEC = "spec"; public static final String SOURCE_METADATA = "metadata"; public static final String SOURCE_DOES_NOT_NEED_SPLITTING = "does_not_need_splitting"; - public static final String SOURCE_PRODUCES_SORTED_KEYS = "produces_sorted_keys"; public static final String SOURCE_IS_INFINITE = "is_infinite"; public static final String SOURCE_ESTIMATED_SIZE_BYTES = "estimated_size_bytes"; - public static final String ELEMENT = "element"; - public static final String ELEMENTS = "elements"; public static final String ENCODING = "encoding"; - public static final String ENCODING_ID = "encoding_id"; - public static final String END_INDEX = "end_index"; - public static final String END_OFFSET = "end_offset"; - public static final String END_SHUFFLE_POSITION = "end_shuffle_position"; public static final String ENVIRONMENT_VERSION_JOB_TYPE_KEY = "job_type"; public static final String ENVIRONMENT_VERSION_MAJOR_KEY = "major"; - public static final String FILENAME = "filename"; - public static final String FILENAME_PREFIX = "filename_prefix"; - public static final String FILENAME_SUFFIX = "filename_suffix"; - public static final String FILEPATTERN = "filepattern"; - public static final String FOOTER = "footer"; public static final String FORMAT = "format"; - public static final String HEADER = "header"; public static final String INPUTS = "inputs"; - public static final String INPUT_CODER = "input_coder"; - public static final String IS_GENERATED = "is_generated"; public static final String IS_MERGING_WINDOW_FN = "is_merging_window_fn"; public static final String IS_PAIR_LIKE = "is_pair_like"; public static final String IS_STREAM_LIKE = "is_stream_like"; public static final String IS_WRAPPER = "is_wrapper"; public static final String DISALLOW_COMBINER_LIFTING = "disallow_combiner_lifting"; public static final String NON_PARALLEL_INPUTS = "non_parallel_inputs"; - public static final String NUM_SHARD_CODERS = "num_shard_coders"; - public static final String NUM_METADATA_SHARD_CODERS = "num_metadata_shard_coders"; - public static final String NUM_SHARDS = "num_shards"; public static final String OBJECT_TYPE_NAME = "@type"; public static final String OUTPUT = "output"; public static final String OUTPUT_INFO = "output_info"; public static final String OUTPUT_NAME = "output_name"; public static final String PARALLEL_INPUT = "parallel_input"; - public static final String PHASE = "phase"; public static final String PUBSUB_ID_ATTRIBUTE = "pubsub_id_label"; public static final String PUBSUB_SERIALIZED_ATTRIBUTES_FN = "pubsub_serialized_attributes_fn"; public static final String PUBSUB_SUBSCRIPTION = "pubsub_subscription"; @@ -91,22 +55,12 @@ public class PropertyNames { public static final String PUBSUB_TOPIC_OVERRIDE = "pubsub_topic_runtime_override"; public static final String SCALAR_FIELD_NAME = "value"; public static final String SERIALIZED_FN = "serialized_fn"; - public static final String SHARD_NAME_TEMPLATE = "shard_template"; - public static final String SHUFFLE_KIND = "shuffle_kind"; - public static final String SHUFFLE_READER_CONFIG = "shuffle_reader_config"; - public static final String SHUFFLE_WRITER_CONFIG = "shuffle_writer_config"; public static final String SORT_VALUES = "sort_values"; - public static final String START_INDEX = "start_index"; - public static final String START_OFFSET = "start_offset"; - public static final String START_SHUFFLE_POSITION = "start_shuffle_position"; - public static final String STRIP_TRAILING_NEWLINES = "strip_trailing_newlines"; public static final String TUPLE_TAGS = "tuple_tags"; public static final String USE_INDEXED_FORMAT = "use_indexed_format"; public static final String USER_FN = "user_fn"; public static final String USER_NAME = "user_name"; public static final String USES_KEYED_STATE = "uses_keyed_state"; - public static final String VALIDATE_SINK = "validate_sink"; - public static final String VALIDATE_SOURCE = "validate_source"; public static final String VALUE = "value"; public static final String DISPLAY_DATA = "display_data"; } From a57ff0eef1b7166d02243e19c4632f8924a014f0 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Thu, 11 May 2017 06:40:47 -0700 Subject: [PATCH 149/159] Dead-letter support for BigQuery. Allow users to specify a retry policy, and get failed inserts back. --- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 8 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 22 ++++- .../sdk/io/gcp/bigquery/BigQueryServices.java | 8 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 39 +++++--- .../io/gcp/bigquery/InsertRetryPolicy.java | 86 ++++++++++++++++++ .../sdk/io/gcp/bigquery/StreamingInserts.java | 35 ++++++-- .../sdk/io/gcp/bigquery/StreamingWriteFn.java | 50 ++++++++--- .../io/gcp/bigquery/StreamingWriteTables.java | 26 ++++-- .../beam/sdk/io/gcp/bigquery/WriteResult.java | 27 ++++-- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 56 ++++++++++++ .../bigquery/BigQueryServicesImplTest.java | 88 ++++++++++++++++--- .../sdk/io/gcp/bigquery/BigQueryUtilTest.java | 11 ++- .../io/gcp/bigquery/FakeDatasetService.java | 75 +++++++++++++++- .../gcp/bigquery/InsertRetryPolicyTest.java | 79 +++++++++++++++++ 14 files changed, 540 insertions(+), 70 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicy.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicyTest.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index 0abd46986f5a..3686f992d882 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -59,6 +59,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.TypeDescriptor; /** PTransform that uses BigQuery batch-load jobs to write a PCollection to BigQuery. */ class BatchLoads @@ -248,7 +249,8 @@ public String apply(String input) { // This transform will look at the set of files written for each table, and if any table has // too many files or bytes, will partition that table's files into multiple partitions for // loading. - PCollection singleton = p.apply(Create.of((Void) null).withCoder(VoidCoder.of())); + PCollection singleton = p.apply("singleton", + Create.of((Void) null).withCoder(VoidCoder.of())); PCollectionTuple partitions = singleton.apply( "WritePartition", @@ -333,6 +335,8 @@ public String apply(String input) { dynamicDestinations)) .withSideInputs(writeTablesSideInputs)); - return WriteResult.in(input.getPipeline()); + PCollection empty = + p.apply("CreateEmptyFailedInserts", Create.empty(TypeDescriptor.of(TableRow.class))); + return WriteResult.in(input.getPipeline(), new TupleTag("failedInserts"), empty); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index cf258ca09f6d..6a93279f7171 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -674,6 +674,7 @@ public abstract static class Write extends PTransform, WriteRe abstract BigQueryServices getBigQueryServices(); @Nullable abstract Integer getMaxFilesPerBundle(); @Nullable abstract Long getMaxFileSize(); + @Nullable abstract InsertRetryPolicy getFailedInsertRetryPolicy(); abstract Builder toBuilder(); @@ -693,6 +694,7 @@ abstract Builder setTableFunction( abstract Builder setBigQueryServices(BigQueryServices bigQueryServices); abstract Builder setMaxFilesPerBundle(Integer maxFilesPerBundle); abstract Builder setMaxFileSize(Long maxFileSize); + abstract Builder setFailedInsertRetryPolicy(InsertRetryPolicy retryPolicy); abstract Write build(); } @@ -861,6 +863,17 @@ public Write withTableDescription(String tableDescription) { return toBuilder().setTableDescription(tableDescription).build(); } + /** Specfies a policy for handling failed inserts. + * + *

Currently this only is allowed when writing an unbounded collection to BigQuery. Bounded + * collections are written using batch load jobs, so we don't get per-element failures. + * Unbounded collections are written using streaming inserts, so we have access to per-element + * insert results. + */ + public Write withFailedInsertRetryPolicy(InsertRetryPolicy retryPolicy) { + return toBuilder().setFailedInsertRetryPolicy(retryPolicy).build(); + } + /** Disables BigQuery table validation. */ public Write withoutValidation() { return toBuilder().setValidate(false).build(); @@ -935,6 +948,7 @@ public WriteResult expand(PCollection input) { "No more than one of jsonSchema, schemaFromView, or dynamicDestinations may " + "be set"); + DynamicDestinations dynamicDestinations = getDynamicDestinations(); if (dynamicDestinations == null) { if (getJsonTableRef() != null) { @@ -981,10 +995,14 @@ private WriteResult expandTyped( "WriteDisposition.WRITE_TRUNCATE is not supported for an unbounded" + " PCollection."); StreamingInserts streamingInserts = - new StreamingInserts<>(getCreateDisposition(), dynamicDestinations); - streamingInserts.setTestServices(getBigQueryServices()); + new StreamingInserts<>(getCreateDisposition(), dynamicDestinations) + .withInsertRetryPolicy(getFailedInsertRetryPolicy()) + .withTestServices((getBigQueryServices())); return rowsWithDestination.apply(streamingInserts); } else { + checkArgument(getFailedInsertRetryPolicy() == null, + "Record-insert retry policies are not supported when using BigQuery load jobs."); + BatchLoads batchLoads = new BatchLoads<>( getWriteDisposition(), getCreateDisposition(), diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java index 1ae10bc4291b..c06722943e70 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java @@ -33,6 +33,7 @@ import java.util.List; import java.util.NoSuchElementException; import javax.annotation.Nullable; +import org.apache.beam.sdk.values.ValueInSingleWindow; /** An interface for real, mock, or fake implementations of Cloud BigQuery services. */ interface BigQueryServices extends Serializable { @@ -161,9 +162,14 @@ void deleteDataset(String projectId, String datasetId) /** * Inserts {@link TableRow TableRows} with the specified insertIds if not null. * + *

If any insert fail permanently according to the retry policy, those rows are added + * to failedInserts. + * *

Returns the total bytes count of {@link TableRow TableRows}. */ - long insertAll(TableReference ref, List rowList, @Nullable List insertIdList) + long insertAll(TableReference ref, List> rowList, + @Nullable List insertIdList, InsertRetryPolicy retryPolicy, + List> failedInserts) throws IOException, InterruptedException; /** Patch BigQuery {@link Table} description. */ diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 5d5a519abf4d..b14405efaf41 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -69,6 +69,7 @@ import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; import org.apache.beam.sdk.util.Transport; +import org.apache.beam.sdk.values.ValueInSingleWindow; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -656,8 +657,11 @@ public void deleteDataset(String projectId, String datasetId) } @VisibleForTesting - long insertAll(TableReference ref, List rowList, @Nullable List insertIdList, - BackOff backoff, final Sleeper sleeper) throws IOException, InterruptedException { + long insertAll(TableReference ref, List> rowList, + @Nullable List insertIdList, + BackOff backoff, final Sleeper sleeper, InsertRetryPolicy retryPolicy, + List> failedInserts) + throws IOException, InterruptedException { checkNotNull(ref, "ref"); if (executor == null) { this.executor = options.as(GcsOptions.class).getExecutorService(); @@ -671,10 +675,10 @@ long insertAll(TableReference ref, List rowList, @Nullable List allErrors = new ArrayList<>(); // These lists contain the rows to publish. Initially the contain the entire list. // If there are failures, they will contain only the failed rows to be retried. - List rowsToPublish = rowList; + List> rowsToPublish = rowList; List idsToPublish = insertIdList; while (true) { - List retryRows = new ArrayList<>(); + List> retryRows = new ArrayList<>(); List retryIds = (idsToPublish != null) ? new ArrayList() : null; int strideIndex = 0; @@ -686,7 +690,7 @@ long insertAll(TableReference ref, List rowList, @Nullable List strideIndices = new ArrayList<>(); for (int i = 0; i < rowsToPublish.size(); ++i) { - TableRow row = rowsToPublish.get(i); + TableRow row = rowsToPublish.get(i).getValue(); TableDataInsertAllRequest.Rows out = new TableDataInsertAllRequest.Rows(); if (idsToPublish != null) { out.setInsertId(idsToPublish.get(i)); @@ -743,18 +747,23 @@ public List call() throws IOException { try { for (int i = 0; i < futures.size(); i++) { List errors = futures.get(i).get(); - if (errors != null) { - for (TableDataInsertAllResponse.InsertErrors error : errors) { - allErrors.add(error); - if (error.getIndex() == null) { - throw new IOException("Insert failed: " + allErrors); - } + if (errors == null) { + continue; + } + for (TableDataInsertAllResponse.InsertErrors error : errors) { + if (error.getIndex() == null) { + throw new IOException("Insert failed: " + error + ", other errors: " + allErrors); + } - int errorIndex = error.getIndex().intValue() + strideIndices.get(i); + int errorIndex = error.getIndex().intValue() + strideIndices.get(i); + if (retryPolicy.shouldRetry(new InsertRetryPolicy.Context(error))) { + allErrors.add(error); retryRows.add(rowsToPublish.get(errorIndex)); if (retryIds != null) { retryIds.add(idsToPublish.get(errorIndex)); } + } else { + failedInserts.add(rowsToPublish.get(errorIndex)); } } } @@ -793,13 +802,15 @@ public List call() throws IOException { @Override public long insertAll( - TableReference ref, List rowList, @Nullable List insertIdList) + TableReference ref, List> rowList, + @Nullable List insertIdList, + InsertRetryPolicy retryPolicy, List> failedInserts) throws IOException, InterruptedException { return insertAll( ref, rowList, insertIdList, BackOffAdapter.toGcpBackOff( INSERT_BACKOFF_FACTORY.backoff()), - Sleeper.DEFAULT); + Sleeper.DEFAULT, retryPolicy, failedInserts); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicy.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicy.java new file mode 100644 index 000000000000..90a3d0d2a8f9 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicy.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.gcp.bigquery; + +import com.google.api.services.bigquery.model.ErrorProto; +import com.google.api.services.bigquery.model.TableDataInsertAllResponse; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +import java.util.Set; + +/** A retry policy for streaming BigQuery inserts. */ +public abstract class InsertRetryPolicy implements Serializable { + /** + * Contains information about a failed insert. + * + *

Currently only the list of errors returned from BigQuery. In the future this may contain + * more information - e.g. how many times this insert has been retried, and for how long. + */ + public static class Context { + // A list of all errors corresponding to an attempted insert of a single record. + TableDataInsertAllResponse.InsertErrors errors; + + public Context(TableDataInsertAllResponse.InsertErrors errors) { + this.errors = errors; + } + } + + // A list of known persistent errors for which retrying never helps. + static final Set PERSISTENT_ERRORS = + ImmutableSet.of("invalid", "invalidQuery", "notImplemented"); + + /** Return true if this failure should be retried. */ + public abstract boolean shouldRetry(Context context); + + /** Never retry any failures. */ + public static InsertRetryPolicy neverRetry() { + return new InsertRetryPolicy() { + @Override + public boolean shouldRetry(Context context) { + return false; + } + }; + } + + /** Always retry all failures. */ + public static InsertRetryPolicy alwaysRetry() { + return new InsertRetryPolicy() { + @Override + public boolean shouldRetry(Context context) { + return true; + } + }; + } + + /** Retry all failures except for known persistent errors. */ + public static InsertRetryPolicy retryTransientErrors() { + return new InsertRetryPolicy() { + @Override + public boolean shouldRetry(Context context) { + if (context.errors.getErrors() != null) { + for (ErrorProto error : context.errors.getErrors()) { + if (error.getReason() != null && PERSISTENT_ERRORS.contains(error.getReason())) { + return false; + } + } + } + return true; + } + }; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java index 9cb00278e4d8..ba09cb38d8d2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java @@ -35,19 +35,39 @@ public class StreamingInserts private BigQueryServices bigQueryServices; private final CreateDisposition createDisposition; private final DynamicDestinations dynamicDestinations; + private InsertRetryPolicy retryPolicy; /** Constructor. */ - StreamingInserts(CreateDisposition createDisposition, + public StreamingInserts(CreateDisposition createDisposition, DynamicDestinations dynamicDestinations) { + this(createDisposition, dynamicDestinations, new BigQueryServicesImpl(), + InsertRetryPolicy.alwaysRetry()); + } + + /** Constructor. */ + private StreamingInserts(CreateDisposition createDisposition, + DynamicDestinations dynamicDestinations, + BigQueryServices bigQueryServices, + InsertRetryPolicy retryPolicy) { this.createDisposition = createDisposition; this.dynamicDestinations = dynamicDestinations; - this.bigQueryServices = new BigQueryServicesImpl(); + this.bigQueryServices = bigQueryServices; + this.retryPolicy = retryPolicy; } - void setTestServices(BigQueryServices bigQueryServices) { - this.bigQueryServices = bigQueryServices; + /** + * Specify a retry policy for failed inserts. + */ + public StreamingInserts withInsertRetryPolicy(InsertRetryPolicy retryPolicy) { + return new StreamingInserts<>( + createDisposition, dynamicDestinations, bigQueryServices, retryPolicy); } + StreamingInserts withTestServices(BigQueryServices bigQueryServices) { + return new StreamingInserts<>( + createDisposition, dynamicDestinations, bigQueryServices, retryPolicy); } + + @Override protected Coder getDefaultOutputCoder() { return VoidCoder.of(); @@ -58,9 +78,12 @@ public WriteResult expand(PCollection> input) { PCollection> writes = input.apply( "CreateTables", - new CreateTables(createDisposition, dynamicDestinations) + new CreateTables<>(createDisposition, dynamicDestinations) .withTestServices(bigQueryServices)); - return writes.apply(new StreamingWriteTables().withTestServices(bigQueryServices)); + return writes.apply( + new StreamingWriteTables() + .withTestServices(bigQueryServices) + .withInsertRetryPolicy(retryPolicy)); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java index f267976f68dd..63e5bc1ceb4f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java @@ -21,6 +21,7 @@ import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; import java.io.IOException; import java.util.HashMap; import java.util.List; @@ -29,8 +30,11 @@ import org.apache.beam.sdk.metrics.SinkMetrics; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.ValueInSingleWindow; /** * Implementation of DoFn to perform streaming BigQuery write. @@ -40,9 +44,12 @@ class StreamingWriteFn extends DoFn, TableRowInfo>, Void> { private final BigQueryServices bqServices; + private final InsertRetryPolicy retryPolicy; + private final TupleTag failedOutputTag; + /** JsonTableRows to accumulate BigQuery rows in order to batch writes. */ - private transient Map> tableRows; + private transient Map>> tableRows; /** The list of unique ids for each BigQuery table row. */ private transient Map> uniqueIdsForTableRows; @@ -50,8 +57,11 @@ class StreamingWriteFn /** Tracks bytes written, exposed as "ByteCount" Counter. */ private Counter byteCounter = SinkMetrics.bytesWritten(); - StreamingWriteFn(BigQueryServices bqServices) { + StreamingWriteFn(BigQueryServices bqServices, InsertRetryPolicy retryPolicy, + TupleTag failedOutputTag) { this.bqServices = bqServices; + this.retryPolicy = retryPolicy; + this.failedOutputTag = failedOutputTag; } /** Prepares a target BigQuery table. */ @@ -63,27 +73,39 @@ public void startBundle() { /** Accumulates the input into JsonTableRows and uniqueIdsForTableRows. */ @ProcessElement - public void processElement(ProcessContext context) { + public void processElement(ProcessContext context, BoundedWindow window) { String tableSpec = context.element().getKey().getKey(); - List rows = BigQueryHelpers.getOrCreateMapListValue(tableRows, tableSpec); - List uniqueIds = BigQueryHelpers.getOrCreateMapListValue(uniqueIdsForTableRows, - tableSpec); + List> rows = + BigQueryHelpers.getOrCreateMapListValue(tableRows, tableSpec); + List uniqueIds = + BigQueryHelpers.getOrCreateMapListValue(uniqueIdsForTableRows, tableSpec); - rows.add(context.element().getValue().tableRow); + rows.add( + ValueInSingleWindow.of( + context.element().getValue().tableRow, context.timestamp(), window, context.pane())); uniqueIds.add(context.element().getValue().uniqueId); } /** Writes the accumulated rows into BigQuery with streaming API. */ @FinishBundle public void finishBundle(FinishBundleContext context) throws Exception { + List> failedInserts = Lists.newArrayList(); BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); - for (Map.Entry> entry : tableRows.entrySet()) { + for (Map.Entry>> entry : tableRows.entrySet()) { TableReference tableReference = BigQueryHelpers.parseTableSpec(entry.getKey()); - flushRows(tableReference, entry.getValue(), - uniqueIdsForTableRows.get(entry.getKey()), options); + flushRows( + tableReference, + entry.getValue(), + uniqueIdsForTableRows.get(entry.getKey()), + options, + failedInserts); } tableRows.clear(); uniqueIdsForTableRows.clear(); + + for (ValueInSingleWindow row : failedInserts) { + context.output(failedOutputTag, row.getValue(), row.getTimestamp(), row.getWindow()); + } } @Override @@ -95,12 +117,14 @@ public void populateDisplayData(DisplayData.Builder builder) { * Writes the accumulated rows into BigQuery with streaming API. */ private void flushRows(TableReference tableReference, - List tableRows, List uniqueIds, BigQueryOptions options) - throws InterruptedException { + List> tableRows, + List uniqueIds, BigQueryOptions options, + List> failedInserts) + throws InterruptedException { if (!tableRows.isEmpty()) { try { long totalBytes = bqServices.getDatasetService(options).insertAll( - tableReference, tableRows, uniqueIds); + tableReference, tableRows, uniqueIds, retryPolicy, failedInserts); byteCounter.inc(totalBytes); } catch (IOException e) { throw new RuntimeException(e); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java index 886236b5f00d..18b203379c7c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java @@ -28,6 +28,9 @@ 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.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; /** * This transform takes in key-value pairs of {@link TableRow} entries and the @@ -40,17 +43,23 @@ public class StreamingWriteTables extends PTransform< PCollection>, WriteResult> { private BigQueryServices bigQueryServices; + private InsertRetryPolicy retryPolicy; public StreamingWriteTables() { - this(new BigQueryServicesImpl()); + this(new BigQueryServicesImpl(), InsertRetryPolicy.alwaysRetry()); } - private StreamingWriteTables(BigQueryServices bigQueryServices) { + private StreamingWriteTables(BigQueryServices bigQueryServices, InsertRetryPolicy retryPolicy) { this.bigQueryServices = bigQueryServices; + this.retryPolicy = retryPolicy; } StreamingWriteTables withTestServices(BigQueryServices bigQueryServices) { - return new StreamingWriteTables(bigQueryServices); + return new StreamingWriteTables(bigQueryServices, retryPolicy); + } + + StreamingWriteTables withInsertRetryPolicy(InsertRetryPolicy retryPolicy) { + return new StreamingWriteTables(bigQueryServices, retryPolicy); } @Override @@ -77,7 +86,9 @@ public WriteResult expand(PCollection> input) { // different unique ids, this implementation relies on "checkpointing", which is // achieved as a side effect of having StreamingWriteFn immediately follow a GBK, // performed by Reshuffle. - tagged + TupleTag mainOutputTag = new TupleTag<>("mainOutput"); + TupleTag failedInsertsTag = new TupleTag<>("failedInserts"); + PCollectionTuple tuple = tagged .setCoder(KvCoder.of(ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowInfoCoder.of())) .apply(Reshuffle., TableRowInfo>of()) // Put in the global window to ensure that DynamicDestinations side inputs are accessed @@ -87,7 +98,10 @@ public WriteResult expand(PCollection> input) { .triggering(DefaultTrigger.of()).discardingFiredPanes()) .apply("StreamingWrite", ParDo.of( - new StreamingWriteFn(bigQueryServices))); - return WriteResult.in(input.getPipeline()); + new StreamingWriteFn(bigQueryServices, retryPolicy, failedInsertsTag)) + .withOutputTags(mainOutputTag, TupleTagList.of(failedInsertsTag))); + PCollection failedInserts = tuple.get(failedInsertsTag); + failedInserts.setCoder(TableRowJsonCoder.of()); + return WriteResult.in(input.getPipeline(), failedInsertsTag, failedInserts); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java index db0be3a96825..4f6b23e16d6c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java @@ -17,10 +17,12 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -import java.util.Collections; +import com.google.api.services.bigquery.model.TableRow; +import com.google.common.collect.ImmutableMap; import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; @@ -30,23 +32,30 @@ * The result of a {@link BigQueryIO.Write} transform. */ public final class WriteResult implements POutput { - private final Pipeline pipeline; + private final TupleTag failedInsertsTag; + private final PCollection failedInserts; - /** - * Creates a {@link WriteResult} in the given {@link Pipeline}. - */ - static WriteResult in(Pipeline pipeline) { - return new WriteResult(pipeline); + /** Creates a {@link WriteResult} in the given {@link Pipeline}. */ + static WriteResult in( + Pipeline pipeline, TupleTag failedInsertsTag, PCollection failedInserts) { + return new WriteResult(pipeline, failedInsertsTag, failedInserts); } @Override public Map, PValue> expand() { - return Collections.emptyMap(); + return ImmutableMap., PValue>of(failedInsertsTag, failedInserts); } - private WriteResult(Pipeline pipeline) { + private WriteResult( + Pipeline pipeline, TupleTag failedInsertsTag, PCollection failedInserts) { this.pipeline = pipeline; + this.failedInsertsTag = failedInsertsTag; + this.failedInserts = failedInserts; + } + + public PCollection getFailedInserts() { + return failedInserts; } @Override diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java index 5408fd4841b9..04bbac431ae9 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java @@ -35,12 +35,14 @@ import static org.junit.Assert.assertTrue; import com.google.api.client.util.Data; +import com.google.api.services.bigquery.model.ErrorProto; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobStatistics; import com.google.api.services.bigquery.model.JobStatistics2; import com.google.api.services.bigquery.model.JobStatistics4; import com.google.api.services.bigquery.model.JobStatus; import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableDataInsertAllResponse; import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; @@ -126,6 +128,7 @@ import org.apache.beam.sdk.util.WindowedValue; 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.PCollectionView; import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.TupleTag; @@ -601,6 +604,59 @@ private void verifySideInputs() { } } + @Test + public void testRetryPolicy() throws Exception { + BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); + bqOptions.setProject("project-id"); + bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + + FakeDatasetService datasetService = new FakeDatasetService(); + FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() + .withJobService(new FakeJobService()) + .withDatasetService(datasetService); + + datasetService.createDataset("project-id", "dataset-id", "", ""); + + TableRow row1 = new TableRow().set("name", "a").set("number", "1"); + TableRow row2 = new TableRow().set("name", "b").set("number", "2"); + TableRow row3 = new TableRow().set("name", "c").set("number", "3"); + + TableDataInsertAllResponse.InsertErrors ephemeralError = + new TableDataInsertAllResponse.InsertErrors().setErrors( + ImmutableList.of(new ErrorProto().setReason("timeout"))); + TableDataInsertAllResponse.InsertErrors persistentError = + new TableDataInsertAllResponse.InsertErrors().setErrors( + ImmutableList.of(new ErrorProto().setReason("invalidQuery"))); + + datasetService.failOnInsert( + ImmutableMap.>of( + row1, ImmutableList.of(ephemeralError, ephemeralError), + row2, ImmutableList.of(ephemeralError, ephemeralError, persistentError))); + + Pipeline p = TestPipeline.create(bqOptions); + PCollection failedRows = + p.apply(Create.of(row1, row2, row3)) + .setIsBoundedInternal(IsBounded.UNBOUNDED) + .apply(BigQueryIO.writeTableRows().to("project-id:dataset-id.table-id") + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) + .withSchema(new TableSchema().setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("number").setType("INTEGER")))) + .withFailedInsertRetryPolicy(InsertRetryPolicy.retryTransientErrors()) + .withTestServices(fakeBqServices) + .withoutValidation()).getFailedInserts(); + // row2 finally fails with a non-retryable error, so we expect to see it in the collection of + // failed rows. + PAssert.that(failedRows).containsInAnyOrder(row2); + p.run(); + + // Only row1 and row3 were successfully inserted. + assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id"), + containsInAnyOrder(row1, row3)); + + } + @Test public void testWrite() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java index b41490f34a38..f602038273f7 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java @@ -58,6 +58,7 @@ import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.cloud.hadoop.util.RetryBoundedBackOff; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; @@ -67,11 +68,14 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.JobServiceImpl; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.ExpectedLogs; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.BackOffAdapter; import org.apache.beam.sdk.util.FastNanoClockAndSleeper; import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; import org.apache.beam.sdk.util.Transport; +import org.apache.beam.sdk.values.ValueInSingleWindow; import org.joda.time.Duration; import org.junit.Before; import org.junit.Rule; @@ -485,6 +489,11 @@ public void testExecuteWithRetries() throws IOException, InterruptedException { verify(response, times(1)).getContentType(); } + private ValueInSingleWindow wrapTableRow(TableRow row) { + return ValueInSingleWindow.of(row, GlobalWindow.TIMESTAMP_MAX_VALUE, + GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING); + } + /** * Tests that {@link DatasetServiceImpl#insertAll} retries quota rate limited attempts. */ @@ -492,8 +501,8 @@ public void testExecuteWithRetries() throws IOException, InterruptedException { public void testInsertRetry() throws Exception { TableReference ref = new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); - List rows = new ArrayList<>(); - rows.add(new TableRow()); + List> rows = new ArrayList<>(); + rows.add(wrapTableRow(new TableRow())); // First response is 403 rate limited, second response has valid payload. when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); @@ -505,7 +514,8 @@ public void testInsertRetry() throws Exception { DatasetServiceImpl dataService = new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); dataService.insertAll(ref, rows, null, - BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()), new MockSleeper()); + BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()), new MockSleeper(), + InsertRetryPolicy.alwaysRetry(), null); verify(response, times(2)).getStatusCode(); verify(response, times(2)).getContent(); verify(response, times(2)).getContentType(); @@ -524,8 +534,9 @@ public void testInsertRetry() throws Exception { public void testInsertRetrySelectRows() throws Exception { TableReference ref = new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); - List rows = ImmutableList.of( - new TableRow().set("row", "a"), new TableRow().set("row", "b")); + List> rows = ImmutableList.of( + wrapTableRow(new TableRow().set("row", "a")), + wrapTableRow(new TableRow().set("row", "b"))); List insertIds = ImmutableList.of("a", "b"); final TableDataInsertAllResponse bFailed = new TableDataInsertAllResponse() @@ -542,11 +553,11 @@ public void testInsertRetrySelectRows() throws Exception { DatasetServiceImpl dataService = new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); dataService.insertAll(ref, rows, insertIds, - BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()), new MockSleeper()); + BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()), new MockSleeper(), + InsertRetryPolicy.alwaysRetry(), null); verify(response, times(2)).getStatusCode(); verify(response, times(2)).getContent(); verify(response, times(2)).getContentType(); - expectedLogs.verifyInfo("Retrying 1 failed inserts to BigQuery"); } /** @@ -556,7 +567,8 @@ public void testInsertRetrySelectRows() throws Exception { public void testInsertFailsGracefully() throws Exception { TableReference ref = new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); - List rows = ImmutableList.of(new TableRow(), new TableRow()); + List> rows = ImmutableList.of( + wrapTableRow(new TableRow()), wrapTableRow(new TableRow())); final TableDataInsertAllResponse row1Failed = new TableDataInsertAllResponse() .setInsertErrors(ImmutableList.of(new InsertErrors().setIndex(1L))); @@ -584,7 +596,8 @@ public InputStream answer(InvocationOnMock invocation) throws Throwable { // Expect it to fail. try { dataService.insertAll(ref, rows, null, - BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()), new MockSleeper()); + BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()), new MockSleeper(), + InsertRetryPolicy.alwaysRetry(), null); fail(); } catch (IOException e) { assertThat(e, instanceOf(IOException.class)); @@ -606,8 +619,8 @@ public InputStream answer(InvocationOnMock invocation) throws Throwable { public void testInsertDoesNotRetry() throws Throwable { TableReference ref = new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); - List rows = new ArrayList<>(); - rows.add(new TableRow()); + List> rows = new ArrayList<>(); + rows.add(wrapTableRow(new TableRow())); // First response is 403 not-rate-limited, second response has valid payload but should not // be invoked. @@ -625,7 +638,8 @@ public void testInsertDoesNotRetry() throws Throwable { try { dataService.insertAll(ref, rows, null, - BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()), new MockSleeper()); + BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()), new MockSleeper(), + InsertRetryPolicy.alwaysRetry(), null); fail(); } catch (RuntimeException e) { verify(response, times(1)).getStatusCode(); @@ -635,6 +649,56 @@ public void testInsertDoesNotRetry() throws Throwable { } } + /** + * Tests that {@link DatasetServiceImpl#insertAll} uses the supplied {@link InsertRetryPolicy}, + * and returns the list of rows not retried. + */ + @Test + public void testInsertRetryPolicy() throws InterruptedException, IOException { + TableReference ref = + new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); + List> rows = ImmutableList.of( + wrapTableRow(new TableRow()), wrapTableRow(new TableRow())); + + // First time row0 fails with a retryable error, and row1 fails with a persistent error. + final TableDataInsertAllResponse firstFailure = new TableDataInsertAllResponse() + .setInsertErrors(ImmutableList.of( + new InsertErrors().setIndex(0L).setErrors( + ImmutableList.of(new ErrorProto().setReason("timeout"))), + new InsertErrors().setIndex(1L).setErrors( + ImmutableList.of(new ErrorProto().setReason("invalid"))))); + + // Second time there is only one row, which fails with a retryable error. + final TableDataInsertAllResponse secondFialure = new TableDataInsertAllResponse() + .setInsertErrors(ImmutableList.of(new InsertErrors().setIndex(0L).setErrors( + ImmutableList.of(new ErrorProto().setReason("timeout"))))); + + // On the final attempt, no failures are returned. + final TableDataInsertAllResponse allRowsSucceeded = new TableDataInsertAllResponse(); + + when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); + // Always return 200. + when(response.getStatusCode()).thenReturn(200); + when(response.getContentType()).thenReturn(Json.MEDIA_TYPE); + when(response.getStatusCode()).thenReturn(200).thenReturn(200); + + // First fail + when(response.getContent()) + .thenReturn(toStream(firstFailure)) + .thenReturn(toStream(secondFialure)) + .thenReturn(toStream(allRowsSucceeded)); + + DatasetServiceImpl dataService = + new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + + List> failedInserts = Lists.newArrayList(); + dataService.insertAll(ref, rows, null, + BackOffAdapter.toGcpBackOff(TEST_BACKOFF.backoff()), new MockSleeper(), + InsertRetryPolicy.retryTransientErrors(), failedInserts); + assertEquals(1, failedInserts.size()); + expectedLogs.verifyInfo("Retrying 1 failed inserts to BigQuery"); + } + /** A helper to wrap a {@link GenericJson} object in a content stream. */ private static InputStream toStream(GenericJson content) throws IOException { return new ByteArrayInputStream(JacksonFactory.getDefaultInstance().toByteArray(content)); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java index fa84119958ae..43290dc951b1 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java @@ -49,6 +49,9 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.values.ValueInSingleWindow; import org.hamcrest.Matchers; import org.junit.After; import org.junit.Assert; @@ -391,16 +394,18 @@ public void testInsertAll() throws Exception, IOException { .parseTableSpec("project:dataset.table"); DatasetServiceImpl datasetService = new DatasetServiceImpl(mockClient, options, 5); - List rows = new ArrayList<>(); + List> rows = new ArrayList<>(); List ids = new ArrayList<>(); for (int i = 0; i < 25; ++i) { - rows.add(rawRow("foo", 1234)); + rows.add(ValueInSingleWindow.of(rawRow("foo", 1234), GlobalWindow.TIMESTAMP_MAX_VALUE, + GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); ids.add(new String()); } long totalBytes = 0; try { - totalBytes = datasetService.insertAll(ref, rows, ids); + totalBytes = datasetService.insertAll(ref, rows, ids, InsertRetryPolicy.alwaysRetry(), + null); } finally { verifyInsertAll(5); // Each of the 25 rows is 23 bytes: "{f=[{v=foo}, {v=1234}]}" diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java index 5103adb07c8c..6ee53404eb49 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeDatasetService.java @@ -25,9 +25,11 @@ import com.google.api.services.bigquery.model.Dataset; import com.google.api.services.bigquery.model.DatasetReference; import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableDataInsertAllResponse; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import java.io.IOException; import java.io.Serializable; import java.util.HashMap; @@ -36,9 +38,15 @@ import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy.Context; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.values.ValueInSingleWindow; /** A fake dataset service that can be serialized, for use in testReadFromTable. */ class FakeDatasetService implements DatasetService, Serializable { + Map> insertErrors = Maps.newHashMap(); + @Override public Table getTable(TableReference tableRef) throws InterruptedException, IOException { @@ -164,10 +172,24 @@ public void deleteDataset(String projectId, String datasetId) } } + public long insertAll(TableReference ref, List rowList, + @Nullable List insertIdList) + throws IOException, InterruptedException { + List> windowedRows = Lists.newArrayList(); + for (TableRow row : rowList) { + windowedRows.add(ValueInSingleWindow.of(row, GlobalWindow.TIMESTAMP_MAX_VALUE, + GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + } + return insertAll(ref, windowedRows, insertIdList, InsertRetryPolicy.alwaysRetry(), null); + } + @Override public long insertAll( - TableReference ref, List rowList, @Nullable List insertIdList) + TableReference ref, List> rowList, + @Nullable List insertIdList, + InsertRetryPolicy retryPolicy, List> failedInserts) throws IOException, InterruptedException { + Map> insertErrors = getInsertErrors(); synchronized (BigQueryIOTest.tables) { if (insertIdList != null) { assertEquals(rowList.size(), insertIdList.size()); @@ -182,7 +204,21 @@ public long insertAll( TableContainer tableContainer = getTableContainer( ref.getProjectId(), ref.getDatasetId(), ref.getTableId()); for (int i = 0; i < rowList.size(); ++i) { - dataSize += tableContainer.addRow(rowList.get(i), insertIdList.get(i)); + TableRow row = rowList.get(i).getValue(); + List allErrors = insertErrors.get(row); + boolean shouldInsert = true; + if (allErrors != null) { + for (TableDataInsertAllResponse.InsertErrors errors : allErrors) { + if (!retryPolicy.shouldRetry(new Context(errors))) { + shouldInsert = false; + } + } + } + if (shouldInsert) { + dataSize += tableContainer.addRow(row, insertIdList.get(i)); + } else { + failedInserts.add(rowList.get(i)); + } } return dataSize; } @@ -200,6 +236,41 @@ public Table patchTableDescription(TableReference tableReference, } } + /** + * Cause a given {@link TableRow} object to fail when it's inserted. The errors link the list + * will be returned on subsequent retries, and the insert will succeed when the errors run out. + */ + public void failOnInsert( + Map> insertErrors) { + synchronized (BigQueryIOTest.tables) { + for (Map.Entry> entry + : insertErrors.entrySet()) { + List errorStrings = Lists.newArrayList(); + for (TableDataInsertAllResponse.InsertErrors errors : entry.getValue()) { + errorStrings.add(BigQueryHelpers.toJsonString(errors)); + } + this.insertErrors.put(BigQueryHelpers.toJsonString(entry.getKey()), errorStrings); + } + } + } + + Map> getInsertErrors() { + Map> parsedInsertErrors = + Maps.newHashMap(); + synchronized (BigQueryIOTest.tables) { + for (Map.Entry> entry : this.insertErrors.entrySet()) { + TableRow tableRow = BigQueryHelpers.fromJsonString(entry.getKey(), TableRow.class); + List allErrors = Lists.newArrayList(); + for (String errorsString : entry.getValue()) { + allErrors.add(BigQueryHelpers.fromJsonString( + errorsString, TableDataInsertAllResponse.InsertErrors.class)); + } + parsedInsertErrors.put(tableRow, allErrors); + } + } + return parsedInsertErrors; + } + void throwNotFound(String format, Object... args) throws IOException { throw new IOException( new GoogleJsonResponseException.Builder(404, diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicyTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicyTest.java new file mode 100644 index 000000000000..b19835da3749 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/InsertRetryPolicyTest.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.gcp.bigquery; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.google.api.services.bigquery.model.ErrorProto; +import com.google.api.services.bigquery.model.TableDataInsertAllResponse; +import com.google.common.collect.Lists; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy.Context; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link InsertRetryPolicy}. + */ +@RunWith(JUnit4.class) +public class InsertRetryPolicyTest { + @Test + public void testNeverRetry() { + assertFalse(InsertRetryPolicy.neverRetry().shouldRetry( + new Context(new TableDataInsertAllResponse.InsertErrors()))); + } + + @Test + public void testAlwaysRetry() { + assertTrue(InsertRetryPolicy.alwaysRetry().shouldRetry( + new Context(new TableDataInsertAllResponse.InsertErrors()))); + } + + @Test + public void testDontRetryPersistentErrors() { + InsertRetryPolicy policy = InsertRetryPolicy.retryTransientErrors(); + assertTrue(policy.shouldRetry(new Context(generateErrorAmongMany( + 5, "timeout", "unavailable")))); + assertFalse(policy.shouldRetry(new Context(generateErrorAmongMany( + 5, "timeout", "invalid")))); + assertFalse(policy.shouldRetry(new Context(generateErrorAmongMany( + 5, "timeout", "invalidQuery")))); + assertFalse(policy.shouldRetry(new Context(generateErrorAmongMany( + 5, "timeout", "notImplemented")))); + } + + private TableDataInsertAllResponse.InsertErrors generateErrorAmongMany( + int numErrors, String baseReason, String exceptionalReason) { + // The retry policies are expected to search through the entire list of ErrorProtos to determine + // whether to retry. Stick the exceptionalReason in a random position to exercise this. + List errorProtos = Lists.newArrayListWithExpectedSize(numErrors); + int exceptionalPosition = ThreadLocalRandom.current().nextInt(numErrors); + for (int i = 0; i < numErrors; ++i) { + ErrorProto error = new ErrorProto(); + error.setReason((i == exceptionalPosition) ? exceptionalReason : baseReason); + errorProtos.add(error); + } + TableDataInsertAllResponse.InsertErrors errors = new TableDataInsertAllResponse.InsertErrors(); + errors.setErrors(errorProtos); + return errors; + } +} From 1bc84fb5ff4ca087c97da45247f1e445eadc48de Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Tue, 16 May 2017 12:12:01 -0700 Subject: [PATCH 150/159] Allow the Distinct transform to deduplicate elements across panes --- .../spark/SparkRunnerDebuggerTest.java | 2 +- .../apache/beam/sdk/transforms/Distinct.java | 80 ++++++++--- .../beam/sdk/transforms/DistinctTest.java | 130 +++++++++++++++++- 3 files changed, 188 insertions(+), 24 deletions(-) diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerDebuggerTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerDebuggerTest.java index 90097511c45b..64ff98cebfd8 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerDebuggerTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerDebuggerTest.java @@ -142,7 +142,7 @@ public void debugStreamingPipeline() { + "_.mapPartitions(new org.apache.beam.sdk.transforms.Distinct$2())\n" + "_.groupByKey()\n" + "_.map(new org.apache.beam.sdk.transforms.Combine$IterableCombineFn())\n" - + "_.mapPartitions(new org.apache.beam.sdk.transforms.Keys$1())\n" + + "_.mapPartitions(new org.apache.beam.sdk.transforms.Distinct$3())\n" + "_.mapPartitions(new org.apache.beam.sdk.transforms.WithKeys$2())\n" + "_."; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java index 2d08cee932eb..d751dbe011eb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java @@ -17,9 +17,15 @@ */ package org.apache.beam.sdk.transforms; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * {@code Distinct} takes a {@code PCollection} and @@ -59,6 +65,8 @@ */ public class Distinct extends PTransform, PCollection> { + private static final Logger LOG = LoggerFactory.getLogger(Distinct.class); + /** * Returns a {@code Distinct} {@code PTransform}. * @@ -66,7 +74,7 @@ public class Distinct extends PTransform, * {@code PCollection}s */ public static Distinct create() { - return new Distinct(); + return new Distinct<>(); } /** @@ -78,26 +86,48 @@ public static Distinct create() { */ public static WithRepresentativeValues withRepresentativeValueFn( SerializableFunction fn) { - return new WithRepresentativeValues(fn, null); + return new WithRepresentativeValues<>(fn, null); + } + + private static void validateWindowStrategy( + WindowingStrategy strategy) { + if (!strategy.getWindowFn().isNonMerging() + && (!strategy.getTrigger().getClass().equals(DefaultTrigger.class) + || strategy.getAllowedLateness().isLongerThan(Duration.ZERO))) { + throw new UnsupportedOperationException(String.format( + "%s does not support non-merging windowing strategies, except when using the default " + + "trigger and zero allowed lateness.", Distinct.class.getSimpleName())); + } } @Override public PCollection expand(PCollection in) { - return in - .apply("CreateIndex", MapElements.via(new SimpleFunction>() { - @Override - public KV apply(T element) { - return KV.of(element, (Void) null); - } - })) - .apply(Combine.perKey( - new SerializableFunction, Void>() { + validateWindowStrategy(in.getWindowingStrategy()); + PCollection> combined = + in.apply("KeyByElement", MapElements.via( + new SimpleFunction>() { @Override - public Void apply(Iterable iter) { - return null; // ignore input - } + public KV apply(T element) { + return KV.of(element, (Void) null); + } })) - .apply(Keys.create()); + .apply("DropValues", + Combine.perKey( + new SerializableFunction, Void>() { + @Override + public Void apply(Iterable iter) { + return null; // ignore input + } + })); + return combined.apply("ExtractFirstKey", ParDo.of(new DoFn, T>() { + @ProcessElement + public void processElement(ProcessContext c) { + if (c.pane().isFirst()) { + // Only output the key if it's the first time it's been seen. + c.output(c.element().getKey()); + } + } + })); } /** @@ -120,22 +150,32 @@ private WithRepresentativeValues( this.representativeType = representativeType; } + @Override public PCollection expand(PCollection in) { + validateWindowStrategy(in.getWindowingStrategy()); WithKeys withKeys = WithKeys.of(fn); if (representativeType != null) { withKeys = withKeys.withKeyType(representativeType); } - return in - .apply(withKeys) - .apply(Combine.perKey( + PCollection> combined = in + .apply("KeyByRepresentativeValue", withKeys) + .apply("OneValuePerKey", Combine.perKey( new Combine.BinaryCombineFn() { @Override public T apply(T left, T right) { return left; } - })) - .apply(Values.create()); + })); + return combined.apply("KeepFirstPane", ParDo.of(new DoFn, T>() { + @ProcessElement + public void processElement(ProcessContext c) { + // Only output the value if it's the first time it's been seen. + if (c.pane().isFirst()) { + c.output(c.element().getValue()); + } + } + })); } /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java index 17bbed64dc82..b9810c11a103 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java @@ -24,12 +24,25 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.testing.UsesTestStream; import org.apache.beam.sdk.testing.ValidatesRunner; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.joda.time.Duration; +import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -85,9 +98,9 @@ public void testDistinctEmpty() { p.run(); } - private static class Keys implements SerializableFunction, String> { + private static class Keys implements SerializableFunction, T> { @Override - public String apply(KV input) { + public T apply(KV input) { return input.getKey(); } } @@ -118,11 +131,122 @@ public void testDistinctWithRepresentativeValue() { PCollection> input = p.apply(Create.of(strings)); PCollection> output = - input.apply(Distinct.withRepresentativeValueFn(new Keys())); + input.apply(Distinct.withRepresentativeValueFn(new Keys()) + .withRepresentativeType(TypeDescriptor.of(String.class))); PAssert.that(output).satisfies(new Checker()); p.run(); } + + @Rule + public TestPipeline windowedDistinctPipeline = TestPipeline.create(); + + @Test + @Category({ValidatesRunner.class, UsesTestStream.class}) + public void testWindowedDistinct() { + Instant base = new Instant(0); + TestStream values = TestStream.create(StringUtf8Coder.of()) + .advanceWatermarkTo(base) + .addElements( + TimestampedValue.of("k1", base), + TimestampedValue.of("k2", base.plus(Duration.standardSeconds(10))), + TimestampedValue.of("k3", base.plus(Duration.standardSeconds(20))), + TimestampedValue.of("k1", base.plus(Duration.standardSeconds(30))), + TimestampedValue.of("k2", base.plus(Duration.standardSeconds(40))), + TimestampedValue.of("k3", base.plus(Duration.standardSeconds(50))), + TimestampedValue.of("k4", base.plus(Duration.standardSeconds(60))), + TimestampedValue.of("k5", base.plus(Duration.standardSeconds(70))), + TimestampedValue.of("k6", base.plus(Duration.standardSeconds(80)))) + .advanceWatermarkToInfinity(); + + PCollection distinctValues = windowedDistinctPipeline + .apply(values) + .apply(Window.into(FixedWindows.of(Duration.standardSeconds(30)))) + .apply(Distinct.create()); + PAssert.that(distinctValues) + .inWindow(new IntervalWindow(base, base.plus(Duration.standardSeconds(30)))) + .containsInAnyOrder("k1", "k2", "k3"); + PAssert.that(distinctValues) + .inWindow(new IntervalWindow(base.plus( + Duration.standardSeconds(30)), base.plus(Duration.standardSeconds(60)))) + .containsInAnyOrder("k1", "k2", "k3"); + PAssert.that(distinctValues) + .inWindow(new IntervalWindow(base.plus( + Duration.standardSeconds(60)), base.plus(Duration.standardSeconds(90)))) + .containsInAnyOrder("k4", "k5", "k6"); + windowedDistinctPipeline.run(); + } + + @Rule + public TestPipeline triggeredDistinctPipeline = TestPipeline.create(); + + @Test + @Category({ValidatesRunner.class, UsesTestStream.class}) + public void testTriggeredDistinct() { + Instant base = new Instant(0); + TestStream values = TestStream.create(StringUtf8Coder.of()) + .advanceWatermarkTo(base) + .addElements( + TimestampedValue.of("k1", base), + TimestampedValue.of("k2", base.plus(Duration.standardSeconds(10))), + TimestampedValue.of("k3", base.plus(Duration.standardSeconds(20)))) + .advanceProcessingTime(Duration.standardMinutes(1)) + .addElements( + TimestampedValue.of("k1", base.plus(Duration.standardSeconds(30))), + TimestampedValue.of("k2", base.plus(Duration.standardSeconds(40))), + TimestampedValue.of("k3", base.plus(Duration.standardSeconds(50)))) + .advanceWatermarkToInfinity(); + + PCollection distinctValues = triggeredDistinctPipeline + .apply(values) + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(1))) + .triggering(Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane().plusDelayOf( + Duration.standardSeconds(30)))) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()) + .apply(Distinct.create()); + PAssert.that(distinctValues).containsInAnyOrder("k1", "k2", "k3"); + triggeredDistinctPipeline.run(); + } + + @Rule + public TestPipeline triggeredDistinctRepresentativePipeline = TestPipeline.create(); + + @Test + @Category({ValidatesRunner.class, UsesTestStream.class}) + public void testTriggeredDistinctRepresentativeValues() { + Instant base = new Instant(0); + TestStream> values = TestStream.create( + KvCoder.of(VarIntCoder.of(), StringUtf8Coder.of())) + .advanceWatermarkTo(base) + .addElements( + TimestampedValue.of(KV.of(1, "k1"), base), + TimestampedValue.of(KV.of(2, "k2"), base.plus(Duration.standardSeconds(10))), + TimestampedValue.of(KV.of(3, "k3"), base.plus(Duration.standardSeconds(20)))) + .advanceProcessingTime(Duration.standardMinutes(1)) + .addElements( + TimestampedValue.of(KV.of(1, "k1"), base.plus(Duration.standardSeconds(30))), + TimestampedValue.of(KV.of(2, "k2"), base.plus(Duration.standardSeconds(40))), + TimestampedValue.of(KV.of(3, "k3"), base.plus(Duration.standardSeconds(50)))) + .advanceWatermarkToInfinity(); + + PCollection> distinctValues = triggeredDistinctRepresentativePipeline + .apply(values) + .apply(Window.>into(FixedWindows.of(Duration.standardMinutes(1))) + .triggering(Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane().plusDelayOf( + Duration.standardSeconds(30)))) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()) + .apply(Distinct.withRepresentativeValueFn(new Keys()) + .withRepresentativeType(TypeDescriptor.of(Integer.class))); + + + PAssert.that(distinctValues).containsInAnyOrder( + KV.of(1, "k1"), KV.of(2, "k2"), KV.of(3, "k3")); + triggeredDistinctRepresentativePipeline.run(); + } } From 5e5fbed70af5d6ff827266d3db89cd5d8d51f544 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Wed, 10 May 2017 19:49:04 +0800 Subject: [PATCH 151/159] [BEAM-2175] [BEAM-1115] Support for new State and Timer API in Spark batch mode --- runners/spark/pom.xml | 2 - .../spark/translation/MultiDoFnFunction.java | 104 +++++++++++++++++- .../translation/SparkProcessContext.java | 23 +++- .../translation/TransformTranslator.java | 84 +++++++++++--- .../StreamingTransformTranslator.java | 3 +- 5 files changed, 189 insertions(+), 27 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 697f67aadaec..ddb4aca73327 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -77,8 +77,6 @@ org.apache.beam.runners.spark.UsesCheckpointRecovery - org.apache.beam.sdk.testing.UsesStatefulParDo, - org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo, org.apache.beam.sdk.testing.UsesCommittedMetrics, org.apache.beam.sdk.testing.UsesTestStream diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java index 32749122d5b2..23d5b32d4e7c 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java @@ -22,16 +22,24 @@ import com.google.common.collect.Iterators; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.Multimap; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.InMemoryStateInternals; +import org.apache.beam.runners.core.InMemoryTimerInternals; +import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.StepContext; +import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; import org.apache.beam.runners.spark.aggregators.NamedAggregators; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.runners.spark.util.SparkSideInputReader; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; @@ -60,6 +68,7 @@ public class MultiDoFnFunction private final List> additionalOutputTags; private final Map, KV, SideInputBroadcast>> sideInputs; private final WindowingStrategy windowingStrategy; + private final boolean stateful; /** * @param aggAccum The Spark {@link Accumulator} that backs the Beam Aggregators. @@ -70,6 +79,7 @@ public class MultiDoFnFunction * @param additionalOutputTags Additional {@link TupleTag output tags}. * @param sideInputs Side inputs used in this {@link DoFn}. * @param windowingStrategy Input {@link WindowingStrategy}. + * @param stateful Stateful {@link DoFn}. */ public MultiDoFnFunction( Accumulator aggAccum, @@ -80,7 +90,8 @@ public MultiDoFnFunction( TupleTag mainOutputTag, List> additionalOutputTags, Map, KV, SideInputBroadcast>> sideInputs, - WindowingStrategy windowingStrategy) { + WindowingStrategy windowingStrategy, + boolean stateful) { this.aggAccum = aggAccum; this.metricsAccum = metricsAccum; this.stepName = stepName; @@ -90,6 +101,7 @@ public MultiDoFnFunction( this.additionalOutputTags = additionalOutputTags; this.sideInputs = sideInputs; this.windowingStrategy = windowingStrategy; + this.stateful = stateful; } @Override @@ -98,7 +110,35 @@ public Iterable, WindowedValue>> call( DoFnOutputManager outputManager = new DoFnOutputManager(); - DoFnRunner doFnRunner = + final InMemoryTimerInternals timerInternals; + final StepContext context; + // Now only implements the StatefulParDo in Batch mode. + if (stateful) { + Object key = null; + if (iter.hasNext()) { + WindowedValue currentValue = iter.next(); + key = ((KV) currentValue.getValue()).getKey(); + iter = Iterators.concat(Iterators.singletonIterator(currentValue), iter); + } + final InMemoryStateInternals stateInternals = InMemoryStateInternals.forKey(key); + timerInternals = new InMemoryTimerInternals(); + context = new StepContext(){ + @Override + public StateInternals stateInternals() { + return stateInternals; + } + + @Override + public TimerInternals timerInternals() { + return timerInternals; + } + }; + } else { + timerInternals = null; + context = new SparkProcessContext.NoOpStepContext(); + } + + final DoFnRunner doFnRunner = DoFnRunners.simpleRunner( runtimeContext.getPipelineOptions(), doFn, @@ -106,20 +146,72 @@ public Iterable, WindowedValue>> call( outputManager, mainOutputTag, additionalOutputTags, - new SparkProcessContext.NoOpStepContext(), + context, windowingStrategy); DoFnRunnerWithMetrics doFnRunnerWithMetrics = new DoFnRunnerWithMetrics<>(stepName, doFnRunner, metricsAccum); - return new SparkProcessContext<>(doFn, doFnRunnerWithMetrics, outputManager) - .processPartition(iter); + return new SparkProcessContext<>( + doFn, doFnRunnerWithMetrics, outputManager, + stateful ? new TimerDataIterator(timerInternals) : + Collections.emptyIterator()).processPartition(iter); + } + + private static class TimerDataIterator implements Iterator { + + private InMemoryTimerInternals timerInternals; + private boolean hasAdvance; + private TimerInternals.TimerData timerData; + + TimerDataIterator(InMemoryTimerInternals timerInternals) { + this.timerInternals = timerInternals; + } + + @Override + public boolean hasNext() { + + // Advance + if (!hasAdvance) { + try { + // Finish any pending windows by advancing the input watermark to infinity. + timerInternals.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE); + // Finally, advance the processing time to infinity to fire any timers. + timerInternals.advanceProcessingTime(BoundedWindow.TIMESTAMP_MAX_VALUE); + timerInternals.advanceSynchronizedProcessingTime( + BoundedWindow.TIMESTAMP_MAX_VALUE); + } catch (Exception e) { + throw new RuntimeException(e); + } + hasAdvance = true; + } + + // Get timer data + return (timerData = timerInternals.removeNextEventTimer()) != null + || (timerData = timerInternals.removeNextProcessingTimer()) != null + || (timerData = timerInternals.removeNextSynchronizedProcessingTimer()) != null; + } + + @Override + public TimerInternals.TimerData next() { + if (timerData == null) { + throw new NoSuchElementException(); + } else { + return timerData; + } + } + + @Override + public void remove() { + throw new RuntimeException("TimerDataIterator not support remove!"); + } + } private class DoFnOutputManager implements SparkProcessContext.SparkOutputManager, WindowedValue>> { - private final Multimap, WindowedValue> outputs = LinkedListMultimap.create();; + private final Multimap, WindowedValue> outputs = LinkedListMultimap.create(); @Override public void clear() { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java index f4ab7d9b02cb..729eb1c95bb0 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkProcessContext.java @@ -18,16 +18,21 @@ package org.apache.beam.runners.spark.translation; +import static com.google.common.base.Preconditions.checkArgument; + import com.google.common.collect.AbstractIterator; import com.google.common.collect.Lists; import java.util.Iterator; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; @@ -39,15 +44,18 @@ class SparkProcessContext { private final DoFn doFn; private final DoFnRunner doFnRunner; private final SparkOutputManager outputManager; + private Iterator timerDataIterator; SparkProcessContext( DoFn doFn, DoFnRunner doFnRunner, - SparkOutputManager outputManager) { + SparkOutputManager outputManager, + Iterator timerDataIterator) { this.doFn = doFn; this.doFnRunner = doFnRunner; this.outputManager = outputManager; + this.timerDataIterator = timerDataIterator; } Iterable processPartition( @@ -137,6 +145,10 @@ protected OutputT computeNext() { // grab the next element and process it. doFnRunner.processElement(inputIterator.next()); outputIterator = getOutputIterator(); + } else if (timerDataIterator.hasNext()) { + clearOutput(); + fireTimer(timerDataIterator.next()); + outputIterator = getOutputIterator(); } else { // no more input to consume, but finishBundle can produce more output if (!calledFinish) { @@ -152,5 +164,14 @@ protected OutputT computeNext() { } } } + + private void fireTimer( + TimerInternals.TimerData timer) { + StateNamespace namespace = timer.getNamespace(); + checkArgument(namespace instanceof StateNamespaces.WindowNamespace); + BoundedWindow window = ((StateNamespaces.WindowNamespace) namespace).getWindow(); + doFnRunner.onTimer(timer.getTimerId(), window, timer.getTimestamp(), timer.getDomain()); + } + } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 742ea83539e2..64aa35a68fa8 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -21,13 +21,14 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static org.apache.beam.runners.spark.translation.TranslationUtils.rejectSplittable; -import static org.apache.beam.runners.spark.translation.TranslationUtils.rejectStateAndTimers; import com.google.common.base.Optional; +import com.google.common.collect.FluentIterable; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import java.util.Collection; import java.util.Collections; +import java.util.Iterator; import java.util.Map; import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; @@ -52,6 +53,8 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; @@ -347,41 +350,57 @@ public String toNativeString() { private static TransformEvaluator> parDo() { return new TransformEvaluator>() { @Override + @SuppressWarnings("unchecked") public void evaluate( ParDo.MultiOutput transform, EvaluationContext context) { String stepName = context.getCurrentTransform().getFullName(); DoFn doFn = transform.getFn(); rejectSplittable(doFn); - rejectStateAndTimers(doFn); - @SuppressWarnings("unchecked") JavaRDD> inRDD = ((BoundedDataset) context.borrowDataset(transform)).getRDD(); WindowingStrategy windowingStrategy = context.getInput(transform).getWindowingStrategy(); Accumulator aggAccum = AggregatorsAccumulator.getInstance(); Accumulator metricsAccum = MetricsAccumulator.getInstance(); - JavaPairRDD, WindowedValue> all = - inRDD.mapPartitionsToPair( - new MultiDoFnFunction<>( - aggAccum, - metricsAccum, - stepName, - doFn, - context.getRuntimeContext(), - transform.getMainOutputTag(), - transform.getAdditionalOutputTags().getAll(), - TranslationUtils.getSideInputs(transform.getSideInputs(), context), - windowingStrategy)); + + JavaPairRDD, WindowedValue> all; + + DoFnSignature signature = DoFnSignatures.getSignature(transform.getFn().getClass()); + boolean stateful = signature.stateDeclarations().size() > 0 + || signature.timerDeclarations().size() > 0; + + MultiDoFnFunction multiDoFnFunction = new MultiDoFnFunction<>( + aggAccum, + metricsAccum, + stepName, + doFn, + context.getRuntimeContext(), + transform.getMainOutputTag(), + transform.getAdditionalOutputTags().getAll(), + TranslationUtils.getSideInputs(transform.getSideInputs(), context), + windowingStrategy, + stateful); + + if (stateful) { + // Based on the fact that the signature is stateful, DoFnSignatures ensures + // that it is also keyed + all = statefulParDoTransform( + (KvCoder) context.getInput(transform).getCoder(), + windowingStrategy.getWindowFn().windowCoder(), + (JavaRDD) inRDD, + (MultiDoFnFunction) multiDoFnFunction); + } else { + all = inRDD.mapPartitionsToPair(multiDoFnFunction); + } + Map, PValue> outputs = context.getOutputs(transform); if (outputs.size() > 1) { // cache the RDD if we're going to filter it more than once. all.cache(); } for (Map.Entry, PValue> output : outputs.entrySet()) { - @SuppressWarnings("unchecked") JavaPairRDD, WindowedValue> filtered = all.filter(new TranslationUtils.TupleTagFilter(output.getKey())); - @SuppressWarnings("unchecked") // Object is the best we can do since different outputs can have different tags JavaRDD> values = (JavaRDD>) (JavaRDD) filtered.values(); @@ -396,6 +415,37 @@ public String toNativeString() { }; } + private static JavaPairRDD, WindowedValue> statefulParDoTransform( + KvCoder kvCoder, + Coder windowCoder, + JavaRDD>> kvInRDD, + MultiDoFnFunction, OutputT> doFnFunction) { + Coder keyCoder = kvCoder.getKeyCoder(); + + final WindowedValue.WindowedValueCoder wvCoder = WindowedValue.FullWindowedValueCoder.of( + kvCoder.getValueCoder(), windowCoder); + + JavaRDD>>>> groupRDD = + GroupCombineFunctions.groupByKeyOnly(kvInRDD, keyCoder, wvCoder); + + return groupRDD.map(new Function< + WindowedValue>>>, Iterator>>>() { + @Override + public Iterator>> call( + WindowedValue>>> input) throws Exception { + final K key = input.getValue().getKey(); + Iterable> value = input.getValue().getValue(); + return FluentIterable.from(value).transform( + new com.google.common.base.Function, WindowedValue>>() { + @Override + public WindowedValue> apply(WindowedValue windowedValue) { + return windowedValue.withValue(KV.of(key, windowedValue.getValue())); + } + }).iterator(); + } + }).flatMapToPair(doFnFunction); + } + private static TransformEvaluator> readBounded() { return new TransformEvaluator>() { @Override diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index 43f4b756d186..cd5bb3ee5df6 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -413,7 +413,8 @@ public JavaPairRDD, WindowedValue> call( transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), sideInputs, - windowingStrategy)); + windowingStrategy, + false)); } }); Map, PValue> outputs = context.getOutputs(transform); From d17c013240a14b12992cf00f30e5151c7e97f360 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 2 Jun 2017 15:57:01 +0200 Subject: [PATCH 152/159] Fix race condition when outputting pushed-back elements in Flink Runner This affected the Flink Streaming Runner DoFnOperator. The recent fix of emitting pushed-back data when receiving a watermark on the first input put the emission at the end of the method. This can cause the emitted data to become late. The fix is to move the pushed-back element emission to the start of the method. --- .../wrappers/streaming/DoFnOperator.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index e47304690b1c..594fe0e7e55a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -445,6 +445,15 @@ public void processWatermark(Watermark mark) throws Exception { @Override public void processWatermark1(Watermark mark) throws Exception { + // We do the check here because we are guaranteed to at least get the +Inf watermark on the + // main input when the job finishes. + if (currentSideInputWatermark >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { + // this means we will never see any more side input + // we also do the check here because we might have received the side-input MAX watermark + // before receiving any main-input data + emitAllPushedBackData(); + } + if (keyCoder == null) { setCurrentInputWatermark(mark.getTimestamp()); long potentialOutputWatermark = @@ -476,15 +485,6 @@ public void processWatermark1(Watermark mark) throws Exception { } pushbackDoFnRunner.finishBundle(); } - - // We do the check here because we are guaranteed to at least get the +Inf watermark on the - // main input when the job finishes. - if (currentSideInputWatermark >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { - // this means we will never see any more side input - // we also do the check here because we might have received the side-input MAX watermark - // before receiving any main-input data - emitAllPushedBackData(); - } } @Override From fbc6cc59bff93dfcf8676f874870a43eeb228c15 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Fri, 2 Jun 2017 11:31:44 +0800 Subject: [PATCH 153/159] [BEAM-2401] Update Flink Runner to Flink 1.3.0 --- runners/flink/pom.xml | 2 +- .../types/CoderTypeSerializer.java | 55 +++++++++++++++++++ .../types/EncodedValueSerializer.java | 18 +----- .../state/FlinkBroadcastStateInternals.java | 29 +++++----- .../streaming/UnboundedSourceWrapperTest.java | 37 +++++++++++++ 5 files changed, 109 insertions(+), 32 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index fb0a67c58ebc..92f95a0c7f1c 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -31,7 +31,7 @@ jar - 1.2.1 + 1.3.0 diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java index e003119aa96f..bea562e0999d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java @@ -24,7 +24,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; +import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -129,4 +131,57 @@ public boolean canEqual(Object obj) { public int hashCode() { return coder.hashCode(); } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + return new CoderTypeSerializerConfigSnapshot<>(coder); + } + + @Override + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof CoderTypeSerializerConfigSnapshot) { + if (coder.equals(((CoderTypeSerializerConfigSnapshot) configSnapshot).coder)) { + return CompatibilityResult.compatible(); + } + } + return CompatibilityResult.requiresMigration(); + } + + /** + * TypeSerializerConfigSnapshot of CoderTypeSerializer. + */ + public static class CoderTypeSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + private Coder coder; + + public CoderTypeSerializerConfigSnapshot(Coder coder) { + this.coder = coder; + } + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + CoderTypeSerializerConfigSnapshot that = (CoderTypeSerializerConfigSnapshot) o; + + return coder != null ? coder.equals(that.coder) : that.coder == null; + } + + @Override + public int hashCode() { + return coder.hashCode(); + } + } + } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java index c3b979401ff1..c40eb462186e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java @@ -20,13 +20,14 @@ import java.io.IOException; import org.apache.beam.sdk.coders.Coder; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; /** * {@link TypeSerializer} for values that were encoded using a {@link Coder}. */ -public final class EncodedValueSerializer extends TypeSerializer { +public final class EncodedValueSerializer extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; @@ -57,7 +58,6 @@ public int getLength() { return -1; } - @Override public void serialize(byte[] record, DataOutputView target) throws IOException { if (record == null) { @@ -94,18 +94,4 @@ public boolean canEqual(Object obj) { return obj instanceof EncodedValueSerializer; } - @Override - public int hashCode() { - return this.getClass().hashCode(); - } - - @Override - public boolean equals(Object obj) { - return obj instanceof EncodedValueSerializer; - } - - @Override - public TypeSerializer duplicate() { - return this; - } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java index f44e66801e4a..6cc24297e846 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java @@ -49,11 +49,11 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.runtime.state.DefaultOperatorStateBackend; +import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.runtime.state.OperatorStateBackend; /** - * {@link StateInternals} that uses a Flink {@link DefaultOperatorStateBackend} + * {@link StateInternals} that uses a Flink {@link OperatorStateBackend} * to manage the broadcast state. * The state is the same on all parallel instances of the operator. * So we just need store state of operator-0 in OperatorStateBackend. @@ -64,13 +64,12 @@ public class FlinkBroadcastStateInternals implements StateInternals { private int indexInSubtaskGroup; - private final DefaultOperatorStateBackend stateBackend; + private final OperatorStateBackend stateBackend; // stateName -> private Map> stateForNonZeroOperator; public FlinkBroadcastStateInternals(int indexInSubtaskGroup, OperatorStateBackend stateBackend) { - //TODO flink do not yet expose through public API - this.stateBackend = (DefaultOperatorStateBackend) stateBackend; + this.stateBackend = stateBackend; this.indexInSubtaskGroup = indexInSubtaskGroup; if (indexInSubtaskGroup != 0) { stateForNonZeroOperator = new HashMap<>(); @@ -178,10 +177,10 @@ private abstract class AbstractBroadcastState { private String name; private final StateNamespace namespace; private final ListStateDescriptor> flinkStateDescriptor; - private final DefaultOperatorStateBackend flinkStateBackend; + private final OperatorStateStore flinkStateBackend; AbstractBroadcastState( - DefaultOperatorStateBackend flinkStateBackend, + OperatorStateBackend flinkStateBackend, String name, StateNamespace namespace, Coder coder) { @@ -211,7 +210,7 @@ Map getMap() throws Exception { if (result != null) { stateForNonZeroOperator.put(name, result); // we don't need it anymore, must clear it. - flinkStateBackend.getBroadcastOperatorState( + flinkStateBackend.getUnionListState( flinkStateDescriptor).clear(); } } @@ -220,7 +219,7 @@ Map getMap() throws Exception { } Map getMapFromBroadcastState() throws Exception { - ListState> state = flinkStateBackend.getBroadcastOperatorState( + ListState> state = flinkStateBackend.getUnionListState( flinkStateDescriptor); Iterable> iterable = state.get(); Map ret = null; @@ -239,7 +238,7 @@ Map getMapFromBroadcastState() throws Exception { */ void updateMap(Map map) throws Exception { if (indexInSubtaskGroup == 0) { - ListState> state = flinkStateBackend.getBroadcastOperatorState( + ListState> state = flinkStateBackend.getUnionListState( flinkStateDescriptor); state.clear(); if (map.size() > 0) { @@ -304,7 +303,7 @@ private class FlinkBroadcastValueState private final StateTag> address; FlinkBroadcastValueState( - DefaultOperatorStateBackend flinkStateBackend, + OperatorStateBackend flinkStateBackend, StateTag> address, StateNamespace namespace, Coder coder) { @@ -365,7 +364,7 @@ private class FlinkBroadcastBagState extends AbstractBroadcastState> address; FlinkBroadcastBagState( - DefaultOperatorStateBackend flinkStateBackend, + OperatorStateBackend flinkStateBackend, StateTag> address, StateNamespace namespace, Coder coder) { @@ -454,7 +453,7 @@ private class FlinkCombiningState private final Combine.CombineFn combineFn; FlinkCombiningState( - DefaultOperatorStateBackend flinkStateBackend, + OperatorStateBackend flinkStateBackend, StateTag> address, Combine.CombineFn combineFn, StateNamespace namespace, @@ -572,7 +571,7 @@ private class FlinkKeyedCombiningState private final FlinkBroadcastStateInternals flinkStateInternals; FlinkKeyedCombiningState( - DefaultOperatorStateBackend flinkStateBackend, + OperatorStateBackend flinkStateBackend, StateTag> address, Combine.CombineFn combineFn, StateNamespace namespace, @@ -709,7 +708,7 @@ private class FlinkCombiningStateWithContext private final CombineWithContext.Context context; FlinkCombiningStateWithContext( - DefaultOperatorStateBackend flinkStateBackend, + OperatorStateBackend flinkStateBackend, StateTag> address, CombineWithContext.CombineFnWithContext combineFn, StateNamespace namespace, diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java index 500fa664bfa8..e3875bc0515b 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java @@ -55,9 +55,12 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.OutputTag; import org.junit.Test; import org.junit.experimental.runners.Enclosed; import org.junit.runner.RunWith; @@ -127,6 +130,7 @@ public void testReaders() throws Exception { try { sourceOperator.open(); sourceOperator.run(checkpointLock, + new TestStreamStatusMaintainer(), new Output>>>>() { private int count = 0; @@ -134,6 +138,11 @@ public void testReaders() throws Exception { public void emitWatermark(Watermark watermark) { } + @Override + public void collect(OutputTag outputTag, StreamRecord streamRecord) { + collect((StreamRecord) streamRecord); + } + @Override public void emitLatencyMarker(LatencyMarker latencyMarker) { } @@ -215,6 +224,7 @@ public void testRestore() throws Exception { try { sourceOperator.open(); sourceOperator.run(checkpointLock, + new TestStreamStatusMaintainer(), new Output>>>>() { private int count = 0; @@ -222,6 +232,11 @@ public void testRestore() throws Exception { public void emitWatermark(Watermark watermark) { } + @Override + public void collect(OutputTag outputTag, StreamRecord streamRecord) { + collect((StreamRecord) streamRecord); + } + @Override public void emitLatencyMarker(LatencyMarker latencyMarker) { } @@ -293,6 +308,7 @@ public void close() { try { restoredSourceOperator.open(); restoredSourceOperator.run(checkpointLock, + new TestStreamStatusMaintainer(), new Output>>>>() { private int count = 0; @@ -300,6 +316,11 @@ public void close() { public void emitWatermark(Watermark watermark) { } + @Override + public void collect(OutputTag outputTag, StreamRecord streamRecord) { + collect((StreamRecord) streamRecord); + } + @Override public void emitLatencyMarker(LatencyMarker latencyMarker) { } @@ -462,4 +483,20 @@ public List getList() { } + private static final class TestStreamStatusMaintainer implements StreamStatusMaintainer { + StreamStatus currentStreamStatus = StreamStatus.ACTIVE; + + @Override + public void toggleStreamStatus(StreamStatus streamStatus) { + if (!currentStreamStatus.equals(streamStatus)) { + currentStreamStatus = streamStatus; + } + } + + @Override + public StreamStatus getStreamStatus() { + return currentStreamStatus; + } + } + } From b8035ae7ad226bd0261a70fb8e0041e0f07e6dfe Mon Sep 17 00:00:00 2001 From: Pei He Date: Sat, 27 May 2017 14:41:26 +0800 Subject: [PATCH 154/159] Flink runner: specify CheckpointingMode through PipelineOptions. --- .../runners/flink/FlinkPipelineExecutionEnvironment.java | 2 +- .../org/apache/beam/runners/flink/FlinkPipelineOptions.java | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java index 7765a007fb43..98f7c5a9e68c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java @@ -227,7 +227,7 @@ private StreamExecutionEnvironment createStreamExecutionEnvironment() { if (checkpointInterval < 1) { throw new IllegalArgumentException("The checkpoint interval must be positive"); } - flinkStreamEnv.enableCheckpointing(checkpointInterval); + flinkStreamEnv.enableCheckpointing(checkpointInterval, options.getCheckpointingMode()); boolean externalizedCheckpoint = options.isExternalizedCheckpointsEnabled(); boolean retainOnCancellation = options.getRetainExternalizedCheckpointsOnCancellation(); if (externalizedCheckpoint) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index 764fa5ffb98f..ee07abbfa9b0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -26,6 +26,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.streaming.api.CheckpointingMode; /** * Options which can be used to configure a Flink PipelineRunner. @@ -70,6 +71,11 @@ public interface FlinkPipelineOptions Long getCheckpointingInterval(); void setCheckpointingInterval(Long interval); + @Description("The checkpointing mode that defines consistency guarantee.") + @Default.Enum("AT_LEAST_ONCE") + CheckpointingMode getCheckpointingMode(); + void setCheckpointingMode(CheckpointingMode mode); + @Description("Sets the number of times that failed tasks are re-executed. " + "A value of zero effectively disables fault tolerance. A value of -1 indicates " + "that the system default value (as defined in the configuration) should be used.") From 802f10afd5d73ba32ad90ba222f2d80216a18a4d Mon Sep 17 00:00:00 2001 From: Pei He Date: Sat, 27 May 2017 14:59:22 +0800 Subject: [PATCH 155/159] Flink runner: specify checkpointTimeout through PipelineOptions. --- .../runners/flink/FlinkPipelineExecutionEnvironment.java | 2 ++ .../org/apache/beam/runners/flink/FlinkPipelineOptions.java | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java index 98f7c5a9e68c..fe5dd87e92a9 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java @@ -228,6 +228,8 @@ private StreamExecutionEnvironment createStreamExecutionEnvironment() { throw new IllegalArgumentException("The checkpoint interval must be positive"); } flinkStreamEnv.enableCheckpointing(checkpointInterval, options.getCheckpointingMode()); + flinkStreamEnv.getCheckpointConfig().setCheckpointTimeout( + options.getCheckpointTimeoutMillis()); boolean externalizedCheckpoint = options.isExternalizedCheckpointsEnabled(); boolean retainOnCancellation = options.getRetainExternalizedCheckpointsOnCancellation(); if (externalizedCheckpoint) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index ee07abbfa9b0..c255672626b0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -76,6 +76,11 @@ public interface FlinkPipelineOptions CheckpointingMode getCheckpointingMode(); void setCheckpointingMode(CheckpointingMode mode); + @Description("The maximum time that a checkpoint may take before being discarded.") + @Default.Long(20 * 60 * 1000) + Long getCheckpointTimeoutMillis(); + void setCheckpointTimeoutMillis(Long checkpointTimeoutMillis); + @Description("Sets the number of times that failed tasks are re-executed. " + "A value of zero effectively disables fault tolerance. A value of -1 indicates " + "that the system default value (as defined in the configuration) should be used.") From c9aac967bccba5c3e1a0a3e4d84a8def1bfa2581 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Mon, 5 Jun 2017 12:38:30 +0800 Subject: [PATCH 156/159] Update against master changes --- runners/gearpump/pom.xml | 1 + .../gearpump/GearpumpPipelineTranslator.java | 9 +-- .../beam/runners/gearpump/GearpumpRunner.java | 7 +- .../CreatePCollectionViewTranslator.java | 45 ----------- .../ParDoSingleOutputTranslator.java | 75 ------------------- .../translators/TranslationContext.java | 5 +- .../translators/utils/DoFnRunnerFactory.java | 6 +- .../translators/utils/NoOpStepContext.java | 37 +-------- .../translators/utils/TranslatorUtils.java | 1 - .../CreatePCollectionViewTranslatorTest.java | 55 -------------- 10 files changed, 17 insertions(+), 224 deletions(-) delete mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java delete mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoSingleOutputTranslator.java delete mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslatorTest.java diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index d4dade1fbd56..beb775382de6 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -86,6 +86,7 @@ ] + 4 diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java index dc4592c67be0..daf65d9406c7 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -30,11 +30,9 @@ import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.runners.gearpump.translators.CreateGearpumpPCollectionViewTranslator; -import org.apache.beam.runners.gearpump.translators.CreatePCollectionViewTranslator; import org.apache.beam.runners.gearpump.translators.FlattenPCollectionsTranslator; import org.apache.beam.runners.gearpump.translators.GroupByKeyTranslator; import org.apache.beam.runners.gearpump.translators.ParDoMultiOutputTranslator; -import org.apache.beam.runners.gearpump.translators.ParDoSingleOutputTranslator; import org.apache.beam.runners.gearpump.translators.ReadBoundedTranslator; import org.apache.beam.runners.gearpump.translators.ReadUnboundedTranslator; import org.apache.beam.runners.gearpump.translators.TransformTranslator; @@ -73,7 +71,7 @@ * into Gearpump {@link Graph}. */ @SuppressWarnings({"rawtypes", "unchecked"}) -public class GearpumpPipelineTranslator implements Pipeline.PipelineVisitor { +public class GearpumpPipelineTranslator extends Pipeline.PipelineVisitor.Defaults { private static final Logger LOG = LoggerFactory.getLogger( GearpumpPipelineTranslator.class); @@ -89,7 +87,6 @@ public class GearpumpPipelineTranslator implements Pipeline.PipelineVisitor { static { // register TransformTranslators - registerTransformTranslator(ParDo.SingleOutput.class, new ParDoSingleOutputTranslator()); registerTransformTranslator(Read.Unbounded.class, new ReadUnboundedTranslator()); registerTransformTranslator(Read.Bounded.class, new ReadBoundedTranslator()); registerTransformTranslator(GroupByKey.class, new GroupByKeyTranslator()); @@ -97,8 +94,6 @@ public class GearpumpPipelineTranslator implements Pipeline.PipelineVisitor { new FlattenPCollectionsTranslator()); registerTransformTranslator(ParDo.MultiOutput.class, new ParDoMultiOutputTranslator()); registerTransformTranslator(Window.Assign.class, new WindowAssignTranslator()); - registerTransformTranslator(View.CreatePCollectionView.class, - new CreatePCollectionViewTranslator()); registerTransformTranslator(CreateGearpumpPCollectionView.class, new CreateGearpumpPCollectionViewTranslator<>()); } @@ -155,7 +150,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node node) { throw new IllegalStateException( "no translator registered for " + transform); } - translationContext.setCurrentTransform(node); + translationContext.setCurrentTransform(node, getPipeline()); translator.translate(transform, translationContext); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index 6df3f2da5edb..30b19353dc93 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -95,19 +95,22 @@ private ClientContext getClientContext(GearpumpPipelineOptions options, Config c */ private Config registerSerializers(Config config, Map userSerializers) { Map serializers = new HashMap<>(); + serializers.put("org.apache.beam.sdk.util.WindowedValue$ValueInGlobalWindow", ""); serializers.put("org.apache.beam.sdk.util.WindowedValue$TimestampedValueInSingleWindow", ""); + serializers.put("org.apache.beam.sdk.util.WindowedValue$TimestampedValueInGlobalWindow", ""); + serializers.put("org.apache.beam.sdk.util.WindowedValue$TimestampedValueInMultipleWindows", ""); serializers.put("org.apache.beam.sdk.transforms.windowing.PaneInfo", ""); serializers.put("org.apache.beam.sdk.transforms.windowing.PaneInfo$Timing", ""); serializers.put("org.joda.time.Instant", ""); serializers.put("org.apache.beam.sdk.values.KV", ""); serializers.put("org.apache.beam.sdk.transforms.windowing.IntervalWindow", ""); serializers.put("org.apache.beam.sdk.values.TimestampedValue", ""); + if (userSerializers != null && !userSerializers.isEmpty()) { serializers.putAll(userSerializers); } + return config.withValue(GEARPUMP_SERIALIZERS, ConfigValueFactory.fromMap(serializers)); } - - } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java deleted file mode 100644 index da55d705a18f..000000000000 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.gearpump.translators; - -import java.util.List; - -import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; - -/** - * View.CreatePCollectionView bridges input stream to down stream - * transforms. - */ -public class CreatePCollectionViewTranslator implements - TransformTranslator> { - - private static final long serialVersionUID = -2394386873317515748L; - - @Override - public void translate(View.CreatePCollectionView transform, - TranslationContext context) { - JavaStream>> inputStream = - context.getInputStream(context.getInput()); - PCollectionView view = (PCollectionView) context.getOutput(); - context.setOutputStream(view, inputStream); - } -} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoSingleOutputTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoSingleOutputTranslator.java deleted file mode 100644 index 6b0e610e23bf..000000000000 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoSingleOutputTranslator.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.gearpump.translators; - -import java.util.Collection; -import java.util.List; -import java.util.Map; - -import org.apache.beam.runners.gearpump.translators.functions.DoFnFunction; -import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollection; - -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.sdk.values.WindowingStrategy; -import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; - -/** - * {@link ParDo.SingleOutput} is translated to Gearpump flatMap function - * with {@link DoFn} wrapped in {@link DoFnFunction}. - */ -public class ParDoSingleOutputTranslator implements - TransformTranslator> { - - private static final long serialVersionUID = -3413205558160983784L; - private final TupleTag mainOutput = new TupleTag<>(); - private final List> sideOutputs = TupleTagList.empty().getAll(); - - @Override - public void translate(ParDo.SingleOutput transform, TranslationContext context) { - DoFn doFn = transform.getFn(); - PCollection output = (PCollection) context.getOutput(); - WindowingStrategy windowingStrategy = output.getWindowingStrategy(); - - Collection> sideInputs = transform.getSideInputs(); - Map> tagsToSideInputs = - TranslatorUtils.getTagsToSideInputs(sideInputs); - JavaStream> inputStream = context.getInputStream( - context.getInput()); - JavaStream unionStream = - TranslatorUtils.withSideInputStream(context, - inputStream, tagsToSideInputs); - - DoFnFunction doFnFunction = new DoFnFunction<>(context.getPipelineOptions(), - doFn, windowingStrategy, sideInputs, tagsToSideInputs, - mainOutput, sideOutputs); - - JavaStream> outputStream = - TranslatorUtils.toList(unionStream) - .flatMap(doFnFunction, transform.getName()) - .map(new TranslatorUtils.FromRawUnionValue(), "from_RawUnionValue"); - - context.setOutputStream(context.getOutput(), outputStream); - } -} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index c3db04411b15..4090354b0356 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -26,6 +26,7 @@ import java.util.Map; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.values.PValue; @@ -52,8 +53,8 @@ public TranslationContext(JavaStreamApp streamApp, GearpumpPipelineOptions pipel this.pipelineOptions = pipelineOptions; } - public void setCurrentTransform(TransformHierarchy.Node treeNode) { - this.currentTransform = treeNode.toAppliedPTransform(); + public void setCurrentTransform(TransformHierarchy.Node treeNode, Pipeline pipeline) { + this.currentTransform = treeNode.toAppliedPTransform(pipeline); } public GearpumpPipelineOptions getPipelineOptions() { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java index d38f11b5ac33..35cf2b5b33ca 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java @@ -24,11 +24,11 @@ import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; -import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.runners.core.ReadyCheckingSideInputReader; import org.apache.beam.runners.core.SimpleDoFnRunner; import org.apache.beam.runners.core.SimplePushbackSideInputDoFnRunner; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; @@ -48,7 +48,7 @@ public class DoFnRunnerFactory implements Serializable { private final DoFnRunners.OutputManager outputManager; private final TupleTag mainOutputTag; private final List> sideOutputTags; - private final ExecutionContext.StepContext stepContext; + private final StepContext stepContext; private final WindowingStrategy windowingStrategy; public DoFnRunnerFactory( @@ -58,7 +58,7 @@ public DoFnRunnerFactory( DoFnRunners.OutputManager outputManager, TupleTag mainOutputTag, List> sideOutputTags, - ExecutionContext.StepContext stepContext, + StepContext stepContext, WindowingStrategy windowingStrategy) { this.fn = doFn; this.options = pipelineOptions; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java index 64fd615dfe92..b795ed989948 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java @@ -18,47 +18,16 @@ package org.apache.beam.runners.gearpump.translators.utils; -import java.io.IOException; import java.io.Serializable; -import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; /** - * serializable {@link ExecutionContext.StepContext} that basically does nothing. + * serializable {@link StepContext} that basically does nothing. */ -public class NoOpStepContext implements ExecutionContext.StepContext, Serializable { - - @Override - public String getStepName() { - throw new UnsupportedOperationException(); - } - - @Override - public String getTransformName() { - throw new UnsupportedOperationException(); - } - - @Override - public void noteOutput(WindowedValue output) { - } - - @Override - public void noteOutput(TupleTag tag, WindowedValue output) { - - } - - @Override - public void writePCollectionViewData(TupleTag tag, - Iterable> data, - Coder>> dataCoder, W window, Coder windowCoder) throws - IOException { - } +public class NoOpStepContext implements StepContext, Serializable { @Override public StateInternals stateInternals() { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java index b8f0ccb41e7e..999afae8a7f8 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java @@ -74,7 +74,6 @@ public static JavaStream withSideInputStream( for (Map.Entry> tagToSideInput: tagsToSideInputs.entrySet()) { // actually JavaStream>> - // check CreatePCollectionViewTranslator JavaStream> sideInputStream = context.getInputStream( tagToSideInput.getValue()); mainStream = mainStream.merge(sideInputStream.map(new ToRawUnionValue<>( diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslatorTest.java deleted file mode 100644 index 42ff14e60adc..000000000000 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslatorTest.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.runners.gearpump.translators; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PValue; -import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; -import org.junit.Test; - -/** Tests for {@link CreatePCollectionViewTranslator}. */ -public class CreatePCollectionViewTranslatorTest { - - @Test - @SuppressWarnings({"rawtypes", "unchecked"}) - public void testTranslate() { - CreatePCollectionViewTranslator translator = new CreatePCollectionViewTranslator(); - View.CreatePCollectionView> createView = - mock(View.CreatePCollectionView.class); - - JavaStream javaStream = mock(JavaStream.class); - TranslationContext translationContext = mock(TranslationContext.class); - - PValue mockInput = mock(PValue.class); - when(translationContext.getInput()).thenReturn(mockInput); - when(translationContext.getInputStream(mockInput)).thenReturn(javaStream); - - PCollectionView view = mock(PCollectionView.class); - when(translationContext.getOutput()).thenReturn(view); - - translator.translate(createView, translationContext); - verify(translationContext, times(1)).setOutputStream(view, javaStream); - } -} From 3c7e3e6ac5d56855b55c1271d54924d9860ae1a0 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 18 May 2017 13:09:36 -0700 Subject: [PATCH 157/159] Activate Gearpump local-validates-runner-tests in precommit --- runners/gearpump/pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index beb775382de6..7e39a484db4c 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -49,7 +49,8 @@ local-validates-runner-tests - false + + true From 98854d4d01ca526ea4a44dc077d2cfb4cddf9914 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 19 May 2017 09:19:42 +0800 Subject: [PATCH 158/159] Respect WindowFn#getOutputTime in gearpump-runner --- .../gearpump/translators/GroupByKeyTranslator.java | 12 ++++++++---- .../translators/GroupByKeyTranslatorTest.java | 8 ++++---- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index 521f665fca5b..7d944a41580c 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -74,7 +74,7 @@ public void translate(GroupByKey transform, TranslationContext context) { new GearpumpWindowFn(windowFn.isNonMerging()), EventTimeTrigger$.MODULE$, Discarding$.MODULE$), "assign_window") .groupBy(new GroupByFn(inputKeyCoder), parallelism, "group_by_Key_and_Window") - .map(new KeyedByTimestamp(timestampCombiner), "keyed_by_timestamp") + .map(new KeyedByTimestamp(windowFn, timestampCombiner), "keyed_by_timestamp") .fold(new Merge<>(windowFn, timestampCombiner), "merge") .map(new Values(), "values"); @@ -146,17 +146,21 @@ protected static class KeyedByTimestamp extends MapFunction>, KV>>> { + private final WindowFn, BoundedWindow> windowFn; private final TimestampCombiner timestampCombiner; - public KeyedByTimestamp(TimestampCombiner timestampCombiner) { + public KeyedByTimestamp(WindowFn, BoundedWindow> windowFn, + TimestampCombiner timestampCombiner) { + this.windowFn = windowFn; this.timestampCombiner = timestampCombiner; } @Override public KV>> map( WindowedValue> wv) { - Instant timestamp = timestampCombiner.assign( - Iterables.getOnlyElement(wv.getWindows()), wv.getTimestamp()); + BoundedWindow window = Iterables.getOnlyElement(wv.getWindows()); + Instant timestamp = timestampCombiner.assign(window + , windowFn.getOutputTime(wv.getTimestamp(), window)); return KV.of(timestamp, wv); } } diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java index 86b60aa38515..d5b931b78868 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import java.time.Instant; @@ -95,18 +94,19 @@ public static Iterable data() { @Test @SuppressWarnings({"rawtypes", "unchecked"}) public void testKeyedByTimestamp() { + WindowFn slidingWindows = Sessions.withGapDuration(Duration.millis(10)); BoundedWindow window = new IntervalWindow(new org.joda.time.Instant(0), new org.joda.time.Instant(10)); GroupByKeyTranslator.KeyedByTimestamp keyedByTimestamp = - new GroupByKeyTranslator.KeyedByTimestamp(timestampCombiner); + new GroupByKeyTranslator.KeyedByTimestamp(slidingWindows, timestampCombiner); WindowedValue> value = WindowedValue.of( KV.of("key", "val"), org.joda.time.Instant.now(), window, PaneInfo.NO_FIRING); KV>> result = keyedByTimestamp.map(value); org.joda.time.Instant time = - timestampCombiner.assign(Iterables.getOnlyElement(value.getWindows()), - value.getTimestamp()); + timestampCombiner.assign(window, + slidingWindows.getOutputTime(value.getTimestamp(), window)); assertThat(result, equalTo(KV.of(time, value))); } From 7653e7ed6de3d9db822dcd390d2bf70819954fa5 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 7 Jun 2017 14:08:04 +0800 Subject: [PATCH 159/159] Fix side input handling in DoFnFunction --- .../translators/TranslationContext.java | 2 ++ .../translators/functions/DoFnFunction.java | 23 ++++--------------- .../translators/io/GearpumpSource.java | 1 - .../translators/utils/TranslatorUtils.java | 5 ++-- 4 files changed, 8 insertions(+), 23 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index 4090354b0356..64a1e0dd8196 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -68,6 +68,8 @@ public JavaStream getInputStream(PValue input) { public void setOutputStream(PValue output, JavaStream outputStream) { if (!streams.containsKey(output)) { streams.put(output, outputStream); + } else { + throw new RuntimeException("set stream for duplicated output " + output); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index f521d7b0579f..6e4fbebd8892 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -24,7 +24,6 @@ import java.io.Serializable; import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; @@ -66,7 +65,6 @@ public class DoFnFunction extends private transient PushbackSideInputDoFnRunner doFnRunner; private transient SideInputHandler sideInputReader; private transient List> pushedBackValues; - private transient Map, List>>> sideInputValues; private final Collection> sideInputs; private final Map> tagsToSideInputs; private final TupleTag mainOutput; @@ -109,7 +107,6 @@ public void setup() { doFnRunner = doFnRunnerFactory.createRunner(sideInputReader); pushedBackValues = new LinkedList<>(); - sideInputValues = new HashMap<>(); outputManager.setup(mainOutput, sideOutputs); } @@ -132,25 +129,14 @@ public Iterator flatMap(List input } else { // side input PCollectionView sideInput = tagsToSideInputs.get(unionValue.getUnionTag()); - WindowedValue sideInputValue = - (WindowedValue) unionValue.getValue(); - Object value = sideInputValue.getValue(); - if (!(value instanceof Iterable)) { - sideInputValue = sideInputValue.withValue(Lists.newArrayList(value)); - } - if (!sideInputValues.containsKey(sideInput)) { - sideInputValues.put(sideInput, new LinkedList>>()); - } - sideInputValues.get(sideInput).add((WindowedValue>) sideInputValue); + WindowedValue> sideInputValue = + (WindowedValue>) unionValue.getValue(); + sideInputReader.addSideInputValue(sideInput, sideInputValue); } } + for (PCollectionView sideInput: sideInputs) { - if (sideInputValues.containsKey(sideInput)) { - for (WindowedValue> value: sideInputValues.get(sideInput)) { - sideInputReader.addSideInputValue(sideInput, value); - } - } for (WindowedValue value : pushedBackValues) { for (BoundedWindow win: value.getWindows()) { BoundedWindow sideInputWindow = @@ -171,7 +157,6 @@ public Iterator flatMap(List input } pushedBackValues.clear(); Iterables.addAll(pushedBackValues, nextPushedBackValues); - sideInputValues.clear(); doFnRunner.finishBundle(); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index 5e7915128162..60f319dccf8c 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -28,7 +28,6 @@ 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.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java index 999afae8a7f8..282f2613cc94 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java @@ -70,11 +70,10 @@ public static JavaStream withSideInputStream( JavaStream> inputStream, Map> tagsToSideInputs) { JavaStream mainStream = - inputStream.map(new ToRawUnionValue("0"), "map_to_RawUnionValue"); + inputStream.map(new ToRawUnionValue<>("0"), "map_to_RawUnionValue"); for (Map.Entry> tagToSideInput: tagsToSideInputs.entrySet()) { - // actually JavaStream>> - JavaStream> sideInputStream = context.getInputStream( + JavaStream>> sideInputStream = context.getInputStream( tagToSideInput.getValue()); mainStream = mainStream.merge(sideInputStream.map(new ToRawUnionValue<>( tagToSideInput.getKey()), "map_to_RawUnionValue"), "merge_to_MainStream");