From de0fdbc1ee7f233e9ae363c7162171135d3b7403 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Sat, 29 Apr 2017 18:51:21 -0700 Subject: [PATCH 001/387] 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 f4dfbb206382d3ea73881727aa8b0f74eaf98ef4 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 2 May 2017 19:31:22 -0700 Subject: [PATCH 002/387] Annotate internal methods of PCollection --- .../org/apache/beam/sdk/values/PCollection.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java index 034f0de67340..20e5d68946b4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java @@ -22,6 +22,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.CannotProvideCoderException.ReasonCode; import org.apache.beam.sdk.coders.Coder; @@ -330,30 +331,27 @@ public PCollection setTypeDescriptor(TypeDescriptor typeDescriptor) { } /** - * Sets the {@link WindowingStrategy} of this {@link PCollection}. - * - *

For use by primitive transformations only. + * For internal use only; no backwards-compatibility guarantees. */ + @Internal public PCollection setWindowingStrategyInternal(WindowingStrategy windowingStrategy) { this.windowingStrategy = windowingStrategy; return this; } /** - * Sets the {@link PCollection.IsBounded} of this {@link PCollection}. - * - *

For use by internal transformations only. + * For internal use only; no backwards-compatibility guarantees. */ + @Internal public PCollection setIsBoundedInternal(IsBounded isBounded) { this.isBounded = isBounded; return this; } /** - * Creates and returns a new {@link PCollection} for a primitive output. - * - *

For use by primitive transformations only. + * For internal use only; no backwards-compatibility guarantees. */ + @Internal public static PCollection createPrimitiveOutputInternal( Pipeline pipeline, WindowingStrategy windowingStrategy, From c1b26a1b53c334ab171fad60501ba67593fde5d2 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 2 May 2017 19:48:38 -0700 Subject: [PATCH 003/387] Annotate internal pieces of sdks.transforms --- .../sdk/transforms/AppliedPTransform.java | 7 ++-- .../beam/sdk/transforms/CombineFnBase.java | 22 ++++++++---- .../beam/sdk/transforms/Materialization.java | 6 +++- .../beam/sdk/transforms/Materializations.java | 11 ++++-- .../org/apache/beam/sdk/transforms/View.java | 36 ++++++++++++------- .../apache/beam/sdk/transforms/ViewFn.java | 6 +++- 6 files changed, 62 insertions(+), 26 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java index bdb61b834026..4e049a590d7e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java @@ -20,16 +20,14 @@ import com.google.auto.value.AutoValue; import java.util.Map; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.annotations.Internal; 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; /** - * Represents the application of a {@link PTransform} to a specific input to produce - * a specific output. - * - *

For internal use. + * For internal use only; no backwards-compatibility guarantees. * *

Inputs and outputs are stored in their expanded forms, as the condensed form of a composite * {@link PInput} or {@link POutput} is a language-specific concept, and {@link AppliedPTransform} @@ -40,6 +38,7 @@ * @param transform output type * @param transform type */ +@Internal @AutoValue public abstract class AppliedPTransform< InputT extends PInput, OutputT extends POutput, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java index a88109968e35..29990cd60b00 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.lang.reflect.Type; import java.lang.reflect.TypeVariable; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -31,14 +32,19 @@ import org.apache.beam.sdk.values.TypeDescriptor; /** - * This class contains the shared interfaces and abstract classes for different types of combine + * For internal use only; no backwards-compatibility guarantees. + * + *

This class contains the shared interfaces and abstract classes for different types of combine * functions. * *

Users should not implement or extend them directly. */ +@Internal public class CombineFnBase { /** - * A {@code GloballyCombineFn} specifies how to combine a + * For internal use only; no backwards-compatibility guarantees. + * + *

A {@code GloballyCombineFn} specifies how to combine a * collection of input values of type {@code InputT} into a single * output value of type {@code OutputT}. It does this via one or more * intermediate mutable accumulator values of type {@code AccumT}. @@ -50,6 +56,7 @@ public class CombineFnBase { * @param type of mutable accumulator values * @param type of output values */ + @Internal public interface GlobalCombineFn extends Serializable, HasDisplayData { /** @@ -93,16 +100,19 @@ Coder getDefaultOutputCoder(CoderRegistry registry, Coder input } /** - * An abstract {@link GlobalCombineFn} base class shared by - * {@link CombineFn} and {@link CombineFnWithContext}. + * For internal use only; no backwards-compatibility guarantees. * - *

Do not extend this class directly. - * Extends {@link CombineFn} and {@link CombineFnWithContext} instead. + *

An abstract {@link GlobalCombineFn} base class shared by {@link CombineFn} and {@link + * CombineFnWithContext}. + * + *

Do not extend this class directly. Extends {@link CombineFn} and {@link + * CombineFnWithContext} instead. * * @param type of input values * @param type of mutable accumulator values * @param type of output values */ + @Internal abstract static class AbstractGlobalCombineFn implements GlobalCombineFn, Serializable { private static final String INCOMPATIBLE_GLOBAL_WINDOW_ERROR_MESSAGE = diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java index 7cd6256d9f00..6fb8c297f547 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java @@ -18,15 +18,19 @@ package org.apache.beam.sdk.transforms; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.runners.PipelineRunner; /** - * How a view should be physically materialized by a {@link PipelineRunner}. + * For internal use only; no backwards-compatibility guarantees. + * + *

How a view should be physically materialized by a {@link PipelineRunner}. * *

A {@link PipelineRunner} will support some set of materializations, and will reject * {@link ViewFn ViewFns} that require materializations it does not support. See * {@link Materializations} for known implementations. */ +@Internal public interface Materialization { /** * Gets the URN describing this {@link Materialization}. This is a stable, SDK-independent URN diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java index 35925fae5f7a..6e4f83d6f6a1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materializations.java @@ -20,12 +20,16 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.util.WindowedValue; /** - * Utility methods for constructing known {@link Materialization materializations} for a + * For internal use only; no backwards-compatibility guarantees. + * + *

Utility methods for constructing known {@link Materialization materializations} for a * {@link ViewFn}. */ +@Internal public class Materializations { /** * The URN for a {@link Materialization} where the primitive view type is an iterable of fully @@ -36,9 +40,12 @@ public class Materializations { "urn:beam:sideinput:materialization:iterable:0.1"; /** - * A {@link Materialization} where the primitive view type is an iterable of fully specified + * For internal use only; no backwards-compatibility guarantees. + * + *

A {@link Materialization} where the primitive view type is an iterable of fully specified * windowed values. */ + @Internal public static Materialization>> iterable() { return new IterableMaterialization<>(); } 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 b3b8918d411b..d17d423441a9 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 @@ -19,6 +19,7 @@ import java.util.List; import java.util.Map; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.runners.PipelineRunner; @@ -237,11 +238,13 @@ public static AsMultimap asMultimap() { } /** - * Not intended for direct use by pipeline authors; public only so a {@link PipelineRunner} may - * override its behavior. + * For internal use only; no backwards-compatibility guarantees. + * + *

Public only so a {@link PipelineRunner} may override its behavior. * *

See {@link View#asList()}. */ + @Internal public static class AsList extends PTransform, PCollectionView>> { private AsList() { } @@ -259,11 +262,13 @@ public PCollectionView> expand(PCollection input) { } /** - * Not intended for direct use by pipeline authors; public only so a {@link PipelineRunner} may - * override its behavior. + * For internal use only; no backwards-compatibility guarantees. + * + *

Public only so a {@link PipelineRunner} may override its behavior. * *

See {@link View#asIterable()}. */ + @Internal public static class AsIterable extends PTransform, PCollectionView>> { private AsIterable() { } @@ -282,11 +287,13 @@ public PCollectionView> expand(PCollection input) { } /** - * Not intended for direct use by pipeline authors; public only so a {@link PipelineRunner} may - * override its behavior. + * For internal use only; no backwards-compatibility guarantees. + * + *

Public only so a {@link PipelineRunner} may override its behavior. * *

See {@link View#asSingleton()}. */ + @Internal public static class AsSingleton extends PTransform, PCollectionView> { private final T defaultValue; private final boolean hasDefault; @@ -396,11 +403,13 @@ public T identity() { } /** - * Not intended for direct use by pipeline authors; public only so a {@link PipelineRunner} may - * override its behavior. + * For internal use only; no backwards-compatibility guarantees. + * + *

Public only so a {@link PipelineRunner} may override its behavior. * *

See {@link View#asMultimap()}. */ + @Internal public static class AsMultimap extends PTransform>, PCollectionView>>> { private AsMultimap() { } @@ -422,11 +431,13 @@ public PCollectionView>> expand(PCollection> input) } /** - * Not intended for direct use by pipeline authors; public only so a {@link PipelineRunner} may - * override its behavior. + * For internal use only; no backwards-compatibility guarantees. + * + *

Public only so a {@link PipelineRunner} may override its behavior. * *

See {@link View#asMap()}. */ + @Internal public static class AsMap extends PTransform>, PCollectionView>> { private AsMap() { } @@ -459,13 +470,14 @@ public PCollectionView> expand(PCollection> input) { // Internal details below /** - * Creates a primitive {@link PCollectionView}. + * For internal use only; no backwards-compatibility guarantees. * - *

For internal use only by runner implementors. + *

Creates a primitive {@link PCollectionView}. * * @param The type of the elements of the input PCollection * @param The type associated with the {@link PCollectionView} used as a side input */ + @Internal public static class CreatePCollectionView extends PTransform, PCollectionView> { private PCollectionView view; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java index cdfcb88652dc..d51a9171035f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java @@ -18,11 +18,14 @@ package org.apache.beam.sdk.transforms; import java.io.Serializable; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; /** - * A function to adapt a primitive "view" of a {@link PCollection} - some materialization + * For internal use only; no backwards-compatibility guarantees. + * + *

A function to adapt a primitive "view" of a {@link PCollection} - some materialization * specified in the Beam model and implemented by the runner - to a user-facing view type * for side input. * @@ -36,6 +39,7 @@ * @param the type of the underlying primitive view, provided by the runner * {@code } the type of the value(s) accessible via this {@link PCollectionView} */ +@Internal public abstract class ViewFn implements Serializable { /** * Gets the materialization of this {@link ViewFn}. From 49cf433c5c08f3cc91512aa9544a36a5d3e84333 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 2 May 2017 19:59:32 -0700 Subject: [PATCH 004/387] Tighten access control and internal annotations for triggers --- .../sdk/transforms/windowing/AfterAll.java | 4 +- .../sdk/transforms/windowing/AfterEach.java | 2 +- .../sdk/transforms/windowing/AfterFirst.java | 2 +- .../sdk/transforms/windowing/AfterPane.java | 2 +- .../windowing/AfterProcessingTime.java | 2 +- .../transforms/windowing/AfterWatermark.java | 4 +- .../transforms/windowing/DefaultTrigger.java | 2 +- .../beam/sdk/transforms/windowing/Never.java | 2 +- .../windowing/OrFinallyTrigger.java | 2 +- .../sdk/transforms/windowing/Repeatedly.java | 2 +- .../windowing/TimestampTransform.java | 41 +++++++++++++++---- .../sdk/transforms/windowing/Trigger.java | 18 ++++++-- 12 files changed, 62 insertions(+), 21 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java index 2747311dff98..eb0a7acbcac6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.List; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.joda.time.Instant; @@ -51,6 +52,7 @@ public static AfterAll of(List triggers) { return new AfterAll(triggers); } + @Internal @Override public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) { // This trigger will fire after the latest of its sub-triggers. @@ -65,7 +67,7 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) { } @Override - public OnceTrigger getContinuationTrigger(List continuationTriggers) { + protected OnceTrigger getContinuationTrigger(List continuationTriggers) { return new AfterAll(continuationTriggers); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java index 56a9d14a2487..1fc4fbf52a97 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java @@ -72,7 +72,7 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) { } @Override - public Trigger getContinuationTrigger(List continuationTriggers) { + protected Trigger getContinuationTrigger(List continuationTriggers) { return Repeatedly.forever(new AfterFirst(continuationTriggers)); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java index 79fd6391d60a..f0beb0a6d82b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java @@ -66,7 +66,7 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) { } @Override - public OnceTrigger getContinuationTrigger(List continuationTriggers) { + protected OnceTrigger getContinuationTrigger(List continuationTriggers) { return new AfterFirst(continuationTriggers); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java index 25c55936a223..eade95d1350a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java @@ -61,7 +61,7 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) { } @Override - public OnceTrigger getContinuationTrigger(List continuationTriggers) { + protected OnceTrigger getContinuationTrigger(List continuationTriggers) { return AfterPane.elementCountAtLeast(1); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java index eda269adf248..cc7ec137b87f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java @@ -40,7 +40,7 @@ public class AfterProcessingTime extends OnceTrigger { private final List timestampTransforms; - public AfterProcessingTime(List timestampTransforms) { + private AfterProcessingTime(List timestampTransforms) { super(null); this.timestampTransforms = timestampTransforms; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java index 6825ab08d1fc..14a8c98a7787 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java @@ -88,7 +88,7 @@ public OnceTrigger getLateTrigger() { } @SuppressWarnings("unchecked") - public AfterWatermarkEarlyAndLate(OnceTrigger earlyTrigger, OnceTrigger lateTrigger) { + private AfterWatermarkEarlyAndLate(OnceTrigger earlyTrigger, OnceTrigger lateTrigger) { super(lateTrigger == null ? ImmutableList.of(earlyTrigger) : ImmutableList.of(earlyTrigger, lateTrigger)); @@ -178,7 +178,7 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) { } @Override - public FromEndOfWindow getContinuationTrigger(List continuationTriggers) { + protected FromEndOfWindow getContinuationTrigger(List continuationTriggers) { return this; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java index a649b4ff1f51..78f373540fd2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java @@ -51,7 +51,7 @@ public boolean isCompatible(Trigger other) { } @Override - public Trigger getContinuationTrigger(List continuationTriggers) { + protected Trigger getContinuationTrigger(List continuationTriggers) { return this; } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java index 664ae83460e8..6dfeea72067f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java @@ -43,7 +43,7 @@ public static NeverTrigger ever() { * The actual trigger class for {@link Never} triggers. */ public static class NeverTrigger extends OnceTrigger { - protected NeverTrigger() { + private NeverTrigger() { super(null); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java index 1ed9b550ca99..ad0de47e0bbb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java @@ -58,7 +58,7 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) { } @Override - public Trigger getContinuationTrigger(List continuationTriggers) { + protected Trigger getContinuationTrigger(List continuationTriggers) { // Use OrFinallyTrigger instead of AfterFirst because the continuation of ACTUAL // may not be a OnceTrigger. return Repeatedly.forever( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java index 40591e3f8bd4..78b79c74fbab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java @@ -66,7 +66,7 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) { } @Override - public Trigger getContinuationTrigger(List continuationTriggers) { + protected Trigger getContinuationTrigger(List continuationTriggers) { return new Repeatedly(continuationTriggers.get(REPEATED)); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampTransform.java index 5318592f5141..8bdf6ee0d486 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampTransform.java @@ -19,37 +19,59 @@ import com.google.auto.value.AutoValue; import java.io.Serializable; +import org.apache.beam.sdk.annotations.Internal; import org.joda.time.Duration; import org.joda.time.Instant; -/** An abstract description of a standardized transformation on timestamps. */ +/** + * For internal use only; no backwards-compatibility guarantees. + * + *

An abstract description of a standardized transformation on timestamps. + */ +@Internal public abstract class TimestampTransform implements Serializable{ - /** Returns a transform that shifts a timestamp later by {@code delay}. */ + TimestampTransform() {} + + /** + * For internal use only; no backwards-compatibility guarantees. + * + *

Returns a transform that shifts a timestamp later by {@code delay}. + */ + @Internal public static TimestampTransform delay(Duration delay) { return new AutoValue_TimestampTransform_Delay(delay); } /** - * Returns a transform that aligns a timestamp to the next boundary of {@code period}, starting + * For internal use only; no backwards-compatibility guarantees. + * + *

Returns a transform that aligns a timestamp to the next boundary of {@code period}, starting * from {@code offset}. */ + @Internal public static TimestampTransform alignTo(Duration period, Instant offset) { return new AutoValue_TimestampTransform_AlignTo(period, offset); } /** - * Returns a transform that aligns a timestamp to the next boundary of {@code period}, starting + * For internal use only; no backwards-compatibility guarantees. + * + *

Returns a transform that aligns a timestamp to the next boundary of {@code period}, starting * from the start of the epoch. */ + @Internal public static TimestampTransform alignTo(Duration period) { return alignTo(period, new Instant(0)); } /** - * Represents the transform that aligns a timestamp to the next boundary of {@link #getPeriod()} - * start at {@link #getOffset()}. + * For internal use only; no backwards-compatibility guarantees. + * + *

Represents the transform that aligns a timestamp to the next boundary of {@link + * #getPeriod()} start at {@link #getOffset()}. */ + @Internal @AutoValue public abstract static class AlignTo extends TimestampTransform { public abstract Duration getPeriod(); @@ -57,7 +79,12 @@ public abstract static class AlignTo extends TimestampTransform { public abstract Instant getOffset(); } - /** Represents the transform that delays a timestamp by {@link #getDelay()}. */ + /** + * For internal use only; no backwards-compatibility guarantees. + * + *

Represents the transform that delays a timestamp by {@link #getDelay()}. + */ + @Internal @AutoValue public abstract static class Delay extends TimestampTransform { public abstract Duration getDelay(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java index 07d30779b079..519ab6789857 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Objects; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.transforms.GroupByKey; import org.joda.time.Instant; @@ -117,8 +118,10 @@ public Trigger getContinuationTrigger() { protected abstract Trigger getContinuationTrigger(List continuationTriggers); /** - * Returns a bound in event time by which this trigger would have fired at least once for a given - * window had there been input data. + * For internal use only; no backwards-compatibility guarantees. + * + *

Returns a bound in event time by which this trigger would have fired at least once for a + * given window had there been input data. * *

For triggers that do not fire based on the watermark advancing, returns {@link * BoundedWindow#TIMESTAMP_MAX_VALUE}. @@ -126,9 +129,15 @@ public Trigger getContinuationTrigger() { *

This estimate may be used, for example, to determine that there are no elements in a * side-input window, which causes the default value to be used instead. */ + @Internal public abstract Instant getWatermarkThatGuaranteesFiring(BoundedWindow window); - /** Returns whether this performs the same triggering as the given {@link Trigger}. */ + /** + * For internal use only; no backwards-compatibility guarantees. + * + *

Returns whether this performs the same triggering as the given {@link Trigger}. + */ + @Internal public boolean isCompatible(Trigger other) { if (!getClass().equals(other.getClass())) { return false; @@ -208,9 +217,12 @@ public OrFinallyTrigger orFinally(OnceTrigger until) { } /** + * For internal use only; no backwards-compatibility guarantees. + * * {@link Trigger Triggers} that are guaranteed to fire at most once should extend {@link * OnceTrigger} rather than the general {@link Trigger} class to indicate that behavior. */ + @Internal public abstract static class OnceTrigger extends Trigger { protected OnceTrigger(List subTriggers) { super(subTriggers); From 9b8a4e5c4b876d4459c64a9bffee613aeae72fb2 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 2 May 2017 20:05:34 -0700 Subject: [PATCH 005/387] The transforms.reflect package is not for users --- .../org/apache/beam/sdk/transforms/reflect/package-info.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java index 4df5209e6b39..fe2f6b1f8cad 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java @@ -16,8 +16,11 @@ * limitations under the License. */ /** - * Defines reflection-based utilities for analyzing {@link org.apache.beam.sdk.transforms.DoFn}'s + * For internal use only; no backwards-compatibility guarantees. + * + *

Defines reflection-based utilities for analyzing {@link org.apache.beam.sdk.transforms.DoFn}'s * and creating {@link org.apache.beam.sdk.transforms.reflect.DoFnSignature}'s and * {@link org.apache.beam.sdk.transforms.reflect.DoFnInvoker}'s from them. */ package org.apache.beam.sdk.transforms.reflect; + From fe51cc0d1a8aa14adbee81b220f9ca8a442f26fe Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 2 May 2017 20:05:45 -0700 Subject: [PATCH 006/387] Annotate internal-only bits of Java sdk.runners --- .../java/org/apache/beam/sdk/runners/PTransformMatcher.java | 6 +++++- .../org/apache/beam/sdk/runners/PTransformOverride.java | 6 +++++- .../apache/beam/sdk/runners/PTransformOverrideFactory.java | 6 +++++- .../apache/beam/sdk/runners/PipelineRunnerRegistrar.java | 4 ++++ .../org/apache/beam/sdk/runners/TransformHierarchy.java | 6 +++++- 5 files changed, 24 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java index 30dca6d1dcee..6378ecc82f6e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java @@ -20,12 +20,16 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; /** - * Matches applications of {@link PTransform PTransforms}. + * For internal use only; no backwards-compatibility guarantees. + * + *

Matches applications of {@link PTransform PTransforms}. */ +@Internal @Experimental(Kind.CORE_RUNNERS_ONLY) public interface PTransformMatcher { boolean matches(AppliedPTransform application); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverride.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverride.java index 33b9114abd57..2820364d82eb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverride.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverride.java @@ -19,12 +19,16 @@ package org.apache.beam.sdk.runners; import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.transforms.PTransform; /** - * A {@link PTransformMatcher} and associated {@link PTransformOverrideFactory} to replace all + * For internal use only; no backwards-compatibility guarantees. + * + *

A {@link PTransformMatcher} and associated {@link PTransformOverrideFactory} to replace all * matching {@link PTransform PTransforms}. */ +@Internal @AutoValue public abstract class PTransformOverride { public static PTransformOverride of( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java index 786c61c95aa1..a28f303688d6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java @@ -23,6 +23,7 @@ import java.util.Map; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; @@ -32,9 +33,12 @@ import org.apache.beam.sdk.values.TupleTag; /** - * Produces {@link PipelineRunner}-specific overrides of {@link PTransform PTransforms}, and + * For internal use only; no backwards-compatibility guarantees. + * + *

Produces {@link PipelineRunner}-specific overrides of {@link PTransform PTransforms}, and * provides mappings between original and replacement outputs. */ +@Internal @Experimental(Kind.CORE_RUNNERS_ONLY) public interface PTransformOverrideFactory< InputT extends PInput, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java index be95044bb6c5..41fd6f0029ba 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java @@ -19,8 +19,11 @@ import com.google.auto.service.AutoService; import java.util.ServiceLoader; +import org.apache.beam.sdk.annotations.Internal; /** + * For internal use only; no backwards-compatibility guarantees. + * * {@link PipelineRunner} creators have the ability to automatically have their * {@link PipelineRunner} registered with this SDK by creating a {@link ServiceLoader} entry * and a concrete implementation of this interface. @@ -33,6 +36,7 @@ *

It is optional but recommended to use one of the many build time tools such as * {@link AutoService} to generate the necessary META-INF files automatically. */ +@Internal public interface PipelineRunnerRegistrar { /** * Get the set of {@link PipelineRunner PipelineRunners} to register. 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 18bf2e95e8b9..92361942fc8a 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 @@ -36,6 +36,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.runners.PTransformOverrideFactory.ReplacementOutput; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -47,9 +48,12 @@ import org.slf4j.LoggerFactory; /** - * Captures information about a collection of transformations and their + * For internal use only; no backwards-compatibility guarantees. + * + *

Captures information about a collection of transformations and their * associated {@link PValue}s. */ +@Internal public class TransformHierarchy { private static final Logger LOG = LoggerFactory.getLogger(TransformHierarchy.class); From 58298d866fe9d1f4fcaf2ccda3078809f4d55b27 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 3 May 2017 10:10:07 -0700 Subject: [PATCH 007/387] Tighten access in sdk.options --- .../{ValueProviderUtils.java => ValueProviders.java} | 4 ++-- .../org/apache/beam/sdk/options/ValueProviderTest.java | 4 ++-- ...eProviderUtilsTest.java => ValueProvidersTest.java} | 10 +++++----- 3 files changed, 9 insertions(+), 9 deletions(-) rename sdks/java/core/src/main/java/org/apache/beam/sdk/options/{ValueProviderUtils.java => ValueProviders.java} (96%) rename sdks/java/core/src/test/java/org/apache/beam/sdk/options/{ValueProviderUtilsTest.java => ValueProvidersTest.java} (90%) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java similarity index 96% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java index 14a5f2391a66..d034b819f40f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviderUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java @@ -28,8 +28,8 @@ /** * Utilities for working with the {@link ValueProvider} interface. */ -public class ValueProviderUtils { - private ValueProviderUtils() {} +class ValueProviders { + private ValueProviders() {} /** * Given {@code serializedOptions} as a JSON-serialized {@link PipelineOptions}, updates diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java index 383de535299d..9369ae6957c1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java @@ -199,7 +199,7 @@ public void testSerializeDeserializeNoArg() throws Exception { ObjectMapper mapper = new ObjectMapper(); String serializedOptions = mapper.writeValueAsString(submitOptions); - String runnerString = ValueProviderUtils.updateSerializedOptions( + String runnerString = ValueProviders.updateSerializedOptions( serializedOptions, ImmutableMap.of("foo", "quux")); TestOptions runtime = mapper.readValue(runnerString, PipelineOptions.class) .as(TestOptions.class); @@ -218,7 +218,7 @@ public void testSerializeDeserializeWithArg() throws Exception { ObjectMapper mapper = new ObjectMapper(); String serializedOptions = mapper.writeValueAsString(submitOptions); - String runnerString = ValueProviderUtils.updateSerializedOptions( + String runnerString = ValueProviders.updateSerializedOptions( serializedOptions, ImmutableMap.of("foo", "quux")); TestOptions runtime = mapper.readValue(runnerString, PipelineOptions.class) .as(TestOptions.class); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProvidersTest.java similarity index 90% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProvidersTest.java index e09f4ad3af2e..14f86bc279e0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProvidersTest.java @@ -26,9 +26,9 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** Tests for {@link ValueProviderUtils}. */ +/** Tests for {@link ValueProviders}. */ @RunWith(JUnit4.class) -public class ValueProviderUtilsTest { +public class ValueProvidersTest { /** A test interface. */ public interface TestOptions extends PipelineOptions { String getString(); @@ -43,7 +43,7 @@ public void testUpdateSerialize() throws Exception { TestOptions submitOptions = PipelineOptionsFactory.as(TestOptions.class); ObjectMapper mapper = new ObjectMapper(); String serializedOptions = mapper.writeValueAsString(submitOptions); - String updatedOptions = ValueProviderUtils.updateSerializedOptions( + String updatedOptions = ValueProviders.updateSerializedOptions( serializedOptions, ImmutableMap.of("string", "bar")); TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class) .as(TestOptions.class); @@ -56,7 +56,7 @@ public void testUpdateSerializeExistingValue() throws Exception { "--string=baz", "--otherString=quux").as(TestOptions.class); ObjectMapper mapper = new ObjectMapper(); String serializedOptions = mapper.writeValueAsString(submitOptions); - String updatedOptions = ValueProviderUtils.updateSerializedOptions( + String updatedOptions = ValueProviders.updateSerializedOptions( serializedOptions, ImmutableMap.of("string", "bar")); TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class) .as(TestOptions.class); @@ -69,7 +69,7 @@ public void testUpdateSerializeEmptyUpdate() throws Exception { TestOptions submitOptions = PipelineOptionsFactory.as(TestOptions.class); ObjectMapper mapper = new ObjectMapper(); String serializedOptions = mapper.writeValueAsString(submitOptions); - String updatedOptions = ValueProviderUtils.updateSerializedOptions( + String updatedOptions = ValueProviders.updateSerializedOptions( serializedOptions, ImmutableMap.of()); TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class) .as(TestOptions.class); From 362d0be79222ad67f1639d54434c1505ef76752b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 3 May 2017 10:13:15 -0700 Subject: [PATCH 008/387] Annotate internal methods on Pipeline --- .../java/org/apache/beam/sdk/Pipeline.java | 29 ++++++++++++++----- 1 file changed, 22 insertions(+), 7 deletions(-) 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 351e1b8170dd..6b15f0d3f655 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 @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; @@ -190,12 +191,15 @@ public OutputT apply( } /** - * Replaces all nodes that match a {@link PTransformOverride} in this pipeline. Overrides are + * For internal use only; no backwards-compatibility guarantees. + * + *

Replaces all nodes that match a {@link PTransformOverride} in this pipeline. Overrides are * applied in the order they are present within the list. * *

After all nodes are replaced, ensures that no nodes in the updated graph match any of the * overrides. */ + @Internal public void replaceAll(List overrides) { for (PTransformOverride override : overrides) { replace(override); @@ -334,10 +338,12 @@ public void setCoderRegistry(CoderRegistry coderRegistry) { } /** - * A {@link PipelineVisitor} can be passed into - * {@link Pipeline#traverseTopologically} to be called for each of the - * transforms and values in the {@link Pipeline}. + * For internal use only; no backwards-compatibility guarantees. + * + *

A {@link PipelineVisitor} can be passed into {@link Pipeline#traverseTopologically} to be + * called for each of the transforms and values in the {@link Pipeline}. */ + @Internal public interface PipelineVisitor { /** * Called for each composite transform after all topological predecessors have been visited @@ -396,7 +402,9 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { } } /** - * Invokes the {@link PipelineVisitor PipelineVisitor's} + * For internal use only; no backwards-compatibility guarantees. + * + *

Invokes the {@link PipelineVisitor PipelineVisitor's} * {@link PipelineVisitor#visitPrimitiveTransform} and * {@link PipelineVisitor#visitValue} operations on each of this * {@link Pipeline Pipeline's} transform and value nodes, in forward @@ -408,14 +416,18 @@ public void visitValue(PValue value, TransformHierarchy.Node producer) { } * *

Typically invoked by {@link PipelineRunner} subclasses. */ + @Internal public void traverseTopologically(PipelineVisitor visitor) { transforms.visit(visitor); } /** - * Like {@link #applyTransform(String, PInput, PTransform)} but defaulting to the name + * For internal use only; no backwards-compatibility guarantees. + * + *

Like {@link #applyTransform(String, PInput, PTransform)} but defaulting to the name * provided by the {@link PTransform}. */ + @Internal public static OutputT applyTransform(InputT input, PTransform transform) { @@ -423,7 +435,9 @@ OutputT applyTransform(InputT input, } /** - * Applies the given {@code PTransform} to this input {@code InputT} and returns + * For internal use only; no backwards-compatibility guarantees. + * + *

Applies the given {@code PTransform} to this input {@code InputT} and returns * its {@code OutputT}. This uses {@code name} to identify this specific application * of the transform. This name is used in various places, including the monitoring UI, * logging, and to stably identify this application node in the {@link Pipeline} graph during @@ -432,6 +446,7 @@ OutputT applyTransform(InputT input, *

Each {@link PInput} subclass that provides an {@code apply} method should delegate to * this method to ensure proper registration with the {@link PipelineRunner}. */ + @Internal public static OutputT applyTransform(String name, InputT input, PTransform transform) { From 1f1c897264ea7ab050c8644344f6e2648af9ae4a Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Wed, 3 May 2017 17:17:11 -0700 Subject: [PATCH 009/387] [BEAM-2165] Update Apex to support serializing/deserializing custom user types configured via Jackson modules --- runners/apex/pom.xml | 4 + .../utils/SerializablePipelineOptions.java | 15 ++- .../utils/PipelineOptionsTest.java | 98 ++++++++++++++++--- 3 files changed, 99 insertions(+), 18 deletions(-) diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 36252e8c0530..aa4bddf57f76 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -62,6 +62,10 @@ malhar-library ${apex.malhar.version} + + com.fasterxml.jackson.core + jackson-core + com.fasterxml.jackson.core jackson-databind diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java index 1a47ed574e2b..14476b57c26c 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.apex.translation.utils; +import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import java.io.Externalizable; import java.io.IOException; @@ -27,6 +28,7 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * A wrapper to enable serialization of {@link PipelineOptions}. @@ -51,13 +53,13 @@ public ApexPipelineOptions get() { @Override public void writeExternal(ObjectOutput out) throws IOException { - out.writeUTF(new ObjectMapper().writeValueAsString(pipelineOptions)); + out.writeUTF(createMapper().writeValueAsString(pipelineOptions)); } @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { String s = in.readUTF(); - this.pipelineOptions = new ObjectMapper().readValue(s, PipelineOptions.class) + this.pipelineOptions = createMapper().readValue(s, PipelineOptions.class) .as(ApexPipelineOptions.class); if (FILE_SYSTEMS_INTIIALIZED.compareAndSet(false, true)) { @@ -66,4 +68,13 @@ public void readExternal(ObjectInput in) throws IOException, ClassNotFoundExcept } } + /** + * Use an {@link ObjectMapper} configured with any {@link Module}s in the class path allowing + * for user specified configuration injection into the ObjectMapper. This supports user custom + * types on {@link PipelineOptions}. + */ + private static ObjectMapper createMapper() { + return new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + } } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/PipelineOptionsTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/PipelineOptionsTest.java index d5eb9a98d645..118ff996bfa8 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/PipelineOptionsTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/PipelineOptionsTest.java @@ -23,15 +23,25 @@ import com.datatorrent.common.util.FSStorageAgent; import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; import com.esotericsoftware.kryo.serializers.JavaSerializer; - +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.auto.service.AutoService; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; - +import java.io.IOException; import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.junit.BeforeClass; import org.junit.Test; /** @@ -49,36 +59,92 @@ public interface MyOptions extends ApexPipelineOptions { void setTestOption(String value); } - private static class MyOptionsWrapper { - private MyOptionsWrapper() { + private static class OptionsWrapper { + private OptionsWrapper() { this(null); // required for Kryo } - private MyOptionsWrapper(ApexPipelineOptions options) { + private OptionsWrapper(ApexPipelineOptions options) { this.options = new SerializablePipelineOptions(options); } @Bind(JavaSerializer.class) private final SerializablePipelineOptions options; } - private static MyOptions options; - - private static final String[] args = new String[]{"--testOption=nothing"}; + @Test + public void testSerialization() { + OptionsWrapper wrapper = new OptionsWrapper( + PipelineOptionsFactory.fromArgs("--testOption=nothing").as(MyOptions.class)); + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + FSStorageAgent.store(bos, wrapper); - @BeforeClass - public static void beforeTest() { - options = PipelineOptionsFactory.fromArgs(args).as(MyOptions.class); + ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray()); + OptionsWrapper wrapperCopy = (OptionsWrapper) FSStorageAgent.retrieve(bis); + assertNotNull(wrapperCopy.options); + assertEquals("nothing", wrapperCopy.options.get().as(MyOptions.class).getTestOption()); } @Test - public void testSerialization() { - MyOptionsWrapper wrapper = new MyOptionsWrapper(PipelineOptionsTest.options); + public void testSerializationWithUserCustomType() { + OptionsWrapper wrapper = new OptionsWrapper( + PipelineOptionsFactory.fromArgs("--jacksonIncompatible=\"testValue\"") + .as(JacksonIncompatibleOptions.class)); ByteArrayOutputStream bos = new ByteArrayOutputStream(); FSStorageAgent.store(bos, wrapper); ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray()); - MyOptionsWrapper wrapperCopy = (MyOptionsWrapper) FSStorageAgent.retrieve(bis); + OptionsWrapper wrapperCopy = (OptionsWrapper) FSStorageAgent.retrieve(bis); assertNotNull(wrapperCopy.options); - assertEquals("nothing", wrapperCopy.options.get().as(MyOptions.class).getTestOption()); + assertEquals("testValue", + wrapperCopy.options.get().as(JacksonIncompatibleOptions.class) + .getJacksonIncompatible().value); + } + + /** PipelineOptions used to test auto registration of Jackson modules. */ + public interface JacksonIncompatibleOptions extends ApexPipelineOptions { + JacksonIncompatible getJacksonIncompatible(); + void setJacksonIncompatible(JacksonIncompatible value); } + /** A Jackson {@link Module} to test auto-registration of modules. */ + @AutoService(Module.class) + public static class RegisteredTestModule extends SimpleModule { + public RegisteredTestModule() { + super("RegisteredTestModule"); + setMixInAnnotation(JacksonIncompatible.class, JacksonIncompatibleMixin.class); + } + } + + /** A class which Jackson does not know how to serialize/deserialize. */ + public static class JacksonIncompatible { + private final String value; + public JacksonIncompatible(String value) { + this.value = value; + } + } + + /** A Jackson mixin used to add annotations to other classes. */ + @JsonDeserialize(using = JacksonIncompatibleDeserializer.class) + @JsonSerialize(using = JacksonIncompatibleSerializer.class) + public static final class JacksonIncompatibleMixin {} + + /** A Jackson deserializer for {@link JacksonIncompatible}. */ + public static class JacksonIncompatibleDeserializer extends + JsonDeserializer { + + @Override + public JacksonIncompatible deserialize(JsonParser jsonParser, + DeserializationContext deserializationContext) throws IOException, JsonProcessingException { + return new JacksonIncompatible(jsonParser.readValueAs(String.class)); + } + } + + /** A Jackson serializer for {@link JacksonIncompatible}. */ + public static class JacksonIncompatibleSerializer extends JsonSerializer { + + @Override + public void serialize(JacksonIncompatible jacksonIncompatible, JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) throws IOException, JsonProcessingException { + jsonGenerator.writeString(jacksonIncompatible.value); + } + } } From e5729b58330a05e7be510710d0027c004704946b Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Wed, 3 May 2017 17:19:00 -0700 Subject: [PATCH 010/387] [BEAM-2165] Update Dataflow to support serializing/deserializing custom user types configured via Jackson modules This also updates the runner harness and existing tests to use a properly constructed ObjectMapper for PipelineOptions. --- .../dataflow/DataflowPipelineTranslator.java | 13 ++- .../DataflowPipelineTranslatorTest.java | 83 +++++++++++++++++++ .../options/DataflowProfilingOptionsTest.java | 4 +- .../DataflowWorkerLoggingOptionsTest.java | 4 +- .../beam/sdk/options/ValueProviders.java | 4 +- .../apache/beam/sdk/testing/TestPipeline.java | 4 +- .../options/PipelineOptionsFactoryTest.java | 2 +- .../options/ProxyInvocationHandlerTest.java | 4 +- .../beam/sdk/options/ValueProviderTest.java | 19 ++--- .../beam/sdk/options/ValueProvidersTest.java | 19 +++-- .../beam/sdk/testing/TestPipelineTest.java | 6 +- .../options/GoogleApiDebugOptionsTest.java | 8 +- .../org/apache/beam/fn/harness/FnHarness.java | 5 +- .../runners/core/BeamFnDataReadRunner.java | 2 - .../runners/core/BeamFnDataWriteRunner.java | 2 - .../control/ProcessBundleHandlerTest.java | 2 - .../core/BeamFnDataReadRunnerTest.java | 2 - .../core/BeamFnDataWriteRunnerTest.java | 2 - 18 files changed, 143 insertions(+), 42 deletions(-) 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 69b4ecd99be3..e727433eac08 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 @@ -32,6 +32,7 @@ import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.services.dataflow.model.AutoscalingSettings; import com.google.api.services.dataflow.model.DataflowPackage; @@ -72,6 +73,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -90,6 +92,7 @@ import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; @@ -111,6 +114,14 @@ public class DataflowPipelineTranslator { private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineTranslator.class); private static final ObjectMapper MAPPER = new ObjectMapper(); + /** + * Use an {@link ObjectMapper} configured with any {@link Module}s in the class path allowing + * for user specified configuration injection into the ObjectMapper. This supports user custom + * types on {@link PipelineOptions}. + */ + private static final ObjectMapper MAPPER_WITH_MODULES = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + private static byte[] serializeWindowingStrategy(WindowingStrategy windowingStrategy) { try { return WindowingStrategies.toProto(windowingStrategy).toByteArray(); @@ -303,7 +314,7 @@ public Job translate(List packages) { try { environment.setSdkPipelineOptions( - MAPPER.readValue(MAPPER.writeValueAsBytes(options), Map.class)); + MAPPER.readValue(MAPPER_WITH_MODULES.writeValueAsBytes(options), Map.class)); } catch (IOException e) { throw new IllegalArgumentException( "PipelineOptions specified failed to serialize to JSON.", e); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index 41f3c925463f..a6ad8c5ca2fd 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -37,11 +37,23 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.api.services.dataflow.Dataflow; import com.google.api.services.dataflow.model.DataflowPackage; import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.Step; import com.google.api.services.dataflow.model.WorkerPool; +import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -70,6 +82,7 @@ import org.apache.beam.sdk.extensions.gcp.auth.TestCredential; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.Count; @@ -217,6 +230,76 @@ public void testSettingOfSdkPipelineOptions() throws IOException { assertThat(optionsMap, hasEntry("numberOfWorkerHarnessThreads", (Object) 0)); } + /** PipelineOptions used to test auto registration of Jackson modules. */ + public interface JacksonIncompatibleOptions extends PipelineOptions { + JacksonIncompatible getJacksonIncompatible(); + void setJacksonIncompatible(JacksonIncompatible value); + } + + /** A Jackson {@link Module} to test auto-registration of modules. */ + @AutoService(Module.class) + public static class RegisteredTestModule extends SimpleModule { + public RegisteredTestModule() { + super("RegisteredTestModule"); + setMixInAnnotation(JacksonIncompatible.class, JacksonIncompatibleMixin.class); + } + } + + /** A class which Jackson does not know how to serialize/deserialize. */ + public static class JacksonIncompatible { + private final String value; + public JacksonIncompatible(String value) { + this.value = value; + } + } + + /** A Jackson mixin used to add annotations to other classes. */ + @JsonDeserialize(using = JacksonIncompatibleDeserializer.class) + @JsonSerialize(using = JacksonIncompatibleSerializer.class) + public static final class JacksonIncompatibleMixin {} + + /** A Jackson deserializer for {@link JacksonIncompatible}. */ + public static class JacksonIncompatibleDeserializer extends + JsonDeserializer { + + @Override + public JacksonIncompatible deserialize(JsonParser jsonParser, + DeserializationContext deserializationContext) throws IOException, JsonProcessingException { + return new JacksonIncompatible(jsonParser.readValueAs(String.class)); + } + } + + /** A Jackson serializer for {@link JacksonIncompatible}. */ + public static class JacksonIncompatibleSerializer extends JsonSerializer { + + @Override + public void serialize(JacksonIncompatible jacksonIncompatible, JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) throws IOException, JsonProcessingException { + jsonGenerator.writeString(jacksonIncompatible.value); + } + } + + @Test + public void testSettingOfPipelineOptionsWithCustomUserType() throws IOException { + DataflowPipelineOptions options = buildPipelineOptions(); + options.setRunner(DataflowRunner.class); + options.as(JacksonIncompatibleOptions.class).setJacksonIncompatible( + new JacksonIncompatible("userCustomTypeTest")); + + Pipeline p = Pipeline.create(options); + p.traverseTopologically(new RecordingPipelineVisitor()); + Job job = + DataflowPipelineTranslator.fromOptions(options) + .translate( + p, DataflowRunner.fromOptions(options), Collections.emptyList()) + .getJob(); + + Map sdkPipelineOptions = job.getEnvironment().getSdkPipelineOptions(); + assertThat(sdkPipelineOptions, hasKey("options")); + Map optionsMap = (Map) sdkPipelineOptions.get("options"); + assertThat(optionsMap, hasEntry("jacksonIncompatible", (Object) "userCustomTypeTest")); + } + @Test public void testNetworkConfig() throws IOException { final String testNetwork = "test-network"; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java index 4018cbbab1ef..68118a437a1d 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowProfilingOptionsTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.hamcrest.Matchers; import org.junit.Test; import org.junit.runner.RunWith; @@ -33,7 +34,8 @@ @RunWith(JUnit4.class) public class DataflowProfilingOptionsTest { - private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); @Test public void testOptionsObject() throws Exception { diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java index b463dcb86686..b1a5258eb074 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.beam.runners.dataflow.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -32,7 +33,8 @@ /** Tests for {@link DataflowWorkerLoggingOptions}. */ @RunWith(JUnit4.class) public class DataflowWorkerLoggingOptionsTest { - private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); @Rule public ExpectedException expectedException = ExpectedException.none(); @Test diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java index d034b819f40f..e2355eeb98fa 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.Map; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Utilities for working with the {@link ValueProvider} interface. @@ -37,7 +38,8 @@ private ValueProviders() {} */ public static String updateSerializedOptions( String serializedOptions, Map runtimeValues) { - ObjectMapper mapper = new ObjectMapper(); + ObjectMapper mapper = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); ObjectNode root, options; try { root = mapper.readValue(serializedOptions, ObjectNode.class); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index ab8772bf0617..4d0cc2b36423 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -47,6 +47,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.junit.experimental.categories.Category; import org.junit.rules.TestRule; import org.junit.runner.Description; @@ -240,7 +241,8 @@ public static class PipelineRunMissingException extends RuntimeException { static final String PROPERTY_BEAM_TEST_PIPELINE_OPTIONS = "beamTestPipelineOptions"; static final String PROPERTY_USE_DEFAULT_DUMMY_RUNNER = "beamUseDummyRunner"; - private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private Optional enforcement = Optional.absent(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java index 11bd7b90d3f2..76a5f18c0397 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java @@ -1689,7 +1689,7 @@ public void testRegistrationOfJacksonModulesForObjectMapper() throws Exception { } /** PipelineOptions used to test auto registration of Jackson modules. */ - public interface JacksonIncompatibleOptions extends PipelineOptions { + interface JacksonIncompatibleOptions extends PipelineOptions { JacksonIncompatible getJacksonIncompatible(); void setJacksonIncompatible(JacksonIncompatible value); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java index 323c24ce5c77..2c43f57a40fe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java @@ -54,6 +54,7 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; @@ -77,7 +78,8 @@ protected void before() { } }; - private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); /** A test interface with some primitives and objects. */ public interface Simple extends PipelineOptions { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java index 9369ae6957c1..e596cc106d40 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java @@ -32,6 +32,7 @@ import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -41,6 +42,8 @@ /** Tests for {@link ValueProvider}. */ @RunWith(JUnit4.class) public class ValueProviderTest { + private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); @Rule public ExpectedException expectedException = ExpectedException.none(); /** A test interface. */ @@ -118,8 +121,7 @@ public void testDefaultRuntimeProvider() { @Test public void testNoDefaultRuntimeProviderWithOverride() throws Exception { - ObjectMapper mapper = new ObjectMapper(); - TestOptions runtime = mapper.readValue( + TestOptions runtime = MAPPER.readValue( "{ \"options\": { \"foo\": \"quux\" }}", PipelineOptions.class) .as(TestOptions.class); @@ -134,8 +136,7 @@ public void testNoDefaultRuntimeProviderWithOverride() throws Exception { @Test public void testDefaultRuntimeProviderWithOverride() throws Exception { - ObjectMapper mapper = new ObjectMapper(); - TestOptions runtime = mapper.readValue( + TestOptions runtime = MAPPER.readValue( "{ \"options\": { \"bar\": \"quux\" }}", PipelineOptions.class) .as(TestOptions.class); @@ -196,12 +197,11 @@ public void testOptionReturnTypeStatic() { public void testSerializeDeserializeNoArg() throws Exception { TestOptions submitOptions = PipelineOptionsFactory.as(TestOptions.class); assertFalse(submitOptions.getFoo().isAccessible()); - ObjectMapper mapper = new ObjectMapper(); - String serializedOptions = mapper.writeValueAsString(submitOptions); + String serializedOptions = MAPPER.writeValueAsString(submitOptions); String runnerString = ValueProviders.updateSerializedOptions( serializedOptions, ImmutableMap.of("foo", "quux")); - TestOptions runtime = mapper.readValue(runnerString, PipelineOptions.class) + TestOptions runtime = MAPPER.readValue(runnerString, PipelineOptions.class) .as(TestOptions.class); ValueProvider vp = runtime.getFoo(); @@ -215,12 +215,11 @@ public void testSerializeDeserializeWithArg() throws Exception { TestOptions submitOptions = PipelineOptionsFactory.fromArgs("--foo=baz").as(TestOptions.class); assertEquals("baz", submitOptions.getFoo().get()); assertTrue(submitOptions.getFoo().isAccessible()); - ObjectMapper mapper = new ObjectMapper(); - String serializedOptions = mapper.writeValueAsString(submitOptions); + String serializedOptions = MAPPER.writeValueAsString(submitOptions); String runnerString = ValueProviders.updateSerializedOptions( serializedOptions, ImmutableMap.of("foo", "quux")); - TestOptions runtime = mapper.readValue(runnerString, PipelineOptions.class) + TestOptions runtime = MAPPER.readValue(runnerString, PipelineOptions.class) .as(TestOptions.class); ValueProvider vp = runtime.getFoo(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProvidersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProvidersTest.java index 14f86bc279e0..dd4d55b60b16 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProvidersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ValueProvidersTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -29,6 +30,9 @@ /** Tests for {@link ValueProviders}. */ @RunWith(JUnit4.class) public class ValueProvidersTest { + private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + /** A test interface. */ public interface TestOptions extends PipelineOptions { String getString(); @@ -41,11 +45,10 @@ public interface TestOptions extends PipelineOptions { @Test public void testUpdateSerialize() throws Exception { TestOptions submitOptions = PipelineOptionsFactory.as(TestOptions.class); - ObjectMapper mapper = new ObjectMapper(); - String serializedOptions = mapper.writeValueAsString(submitOptions); + String serializedOptions = MAPPER.writeValueAsString(submitOptions); String updatedOptions = ValueProviders.updateSerializedOptions( serializedOptions, ImmutableMap.of("string", "bar")); - TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class) + TestOptions runtime = MAPPER.readValue(updatedOptions, PipelineOptions.class) .as(TestOptions.class); assertEquals("bar", runtime.getString()); } @@ -54,11 +57,10 @@ public void testUpdateSerialize() throws Exception { public void testUpdateSerializeExistingValue() throws Exception { TestOptions submitOptions = PipelineOptionsFactory.fromArgs( "--string=baz", "--otherString=quux").as(TestOptions.class); - ObjectMapper mapper = new ObjectMapper(); - String serializedOptions = mapper.writeValueAsString(submitOptions); + String serializedOptions = MAPPER.writeValueAsString(submitOptions); String updatedOptions = ValueProviders.updateSerializedOptions( serializedOptions, ImmutableMap.of("string", "bar")); - TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class) + TestOptions runtime = MAPPER.readValue(updatedOptions, PipelineOptions.class) .as(TestOptions.class); assertEquals("bar", runtime.getString()); assertEquals("quux", runtime.getOtherString()); @@ -67,11 +69,10 @@ public void testUpdateSerializeExistingValue() throws Exception { @Test public void testUpdateSerializeEmptyUpdate() throws Exception { TestOptions submitOptions = PipelineOptionsFactory.as(TestOptions.class); - ObjectMapper mapper = new ObjectMapper(); - String serializedOptions = mapper.writeValueAsString(submitOptions); + String serializedOptions = MAPPER.writeValueAsString(submitOptions); String updatedOptions = ValueProviders.updateSerializedOptions( serializedOptions, ImmutableMap.of()); - TestOptions runtime = mapper.readValue(updatedOptions, PipelineOptions.class) + TestOptions runtime = MAPPER.readValue(updatedOptions, PipelineOptions.class) .as(TestOptions.class); assertNull(runtime.getString()); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java index 04005c5893b8..05abb59c597e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java @@ -40,6 +40,7 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.PCollection; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; @@ -61,6 +62,8 @@ TestPipelineTest.TestPipelineEnforcementsTest.WithCrashingPipelineRunner.class }) public class TestPipelineTest implements Serializable { + private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); /** Tests related to the creation of a {@link TestPipeline}. */ @RunWith(JUnit4.class) @@ -85,9 +88,8 @@ public void testCreationNotAsTestRule() { @Test public void testCreationOfPipelineOptions() throws Exception { - ObjectMapper mapper = new ObjectMapper(); String stringOptions = - mapper.writeValueAsString( + MAPPER.writeValueAsString( new String[] { "--runner=org.apache.beam.sdk.testing.CrashingRunner" }); diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GoogleApiDebugOptionsTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GoogleApiDebugOptionsTest.java index 67d588029f9d..68a29e68e74a 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GoogleApiDebugOptionsTest.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GoogleApiDebugOptionsTest.java @@ -28,6 +28,7 @@ import org.apache.beam.sdk.extensions.gcp.options.GoogleApiDebugOptions.GoogleApiTracer; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.Transport; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -35,6 +36,8 @@ /** Tests for {@link GoogleApiDebugOptions}. */ @RunWith(JUnit4.class) public class GoogleApiDebugOptionsTest { + private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); private static final String STORAGE_GET_TRACE = "--googleApiTrace={\"Objects.Get\":\"GetTraceDestination\"}"; private static final String STORAGE_GET_AND_LIST_TRACE = @@ -139,9 +142,8 @@ public void testMatchingAgainstRequestType() throws Exception { @Test public void testDeserializationAndSerializationOfGoogleApiTracer() throws Exception { String serializedValue = "{\"Api\":\"Token\"}"; - ObjectMapper objectMapper = new ObjectMapper(); assertEquals(serializedValue, - objectMapper.writeValueAsString( - objectMapper.readValue(serializedValue, GoogleApiTracer.class))); + MAPPER.writeValueAsString( + MAPPER.readValue(serializedValue, GoogleApiTracer.class))); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index d5879864db84..24f826c5c522 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,7 +73,9 @@ public static void main(String[] args) throws Exception { System.out.format("Control location %s%n", System.getenv(CONTROL_API_SERVICE_DESCRIPTOR)); System.out.format("Pipeline options %s%n", System.getenv(PIPELINE_OPTIONS)); - PipelineOptions options = new ObjectMapper().readValue( + ObjectMapper objectMapper = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + PipelineOptions options = objectMapper.readValue( System.getenv(PIPELINE_OPTIONS), PipelineOptions.class); BeamFnApi.ApiServiceDescriptor loggingApiServiceDescriptor = diff --git a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java index 805d480e7dae..7c4a5e8fa272 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.core; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import java.io.IOException; @@ -47,7 +46,6 @@ */ public class BeamFnDataReadRunner { private static final Logger LOG = LoggerFactory.getLogger(BeamFnDataReadRunner.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private final BeamFnApi.ApiServiceDescriptor apiServiceDescriptor; private final Collection>> consumers; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java index 0ba09e392a26..3a11def393fc 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.core; -import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.util.function.Supplier; import org.apache.beam.fn.harness.data.BeamFnDataClient; @@ -38,7 +37,6 @@ * For each request, call {@link #registerForOutput()} to start and call {@link #close()} to finish. */ public class BeamFnDataWriteRunner { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private final BeamFnApi.ApiServiceDescriptor apiServiceDescriptor; private final BeamFnApi.Target outputTarget; private final Coder> coder; 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 59872678b3cd..654f98988cc0 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 @@ -33,7 +33,6 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; @@ -92,7 +91,6 @@ /** Tests for {@link ProcessBundleHandler}. */ @RunWith(JUnit4.class) public class ProcessBundleHandlerTest { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final Coder> STRING_CODER = WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); private static final String LONG_CODER_SPEC_ID = "998L"; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java index 0d036fe4edc7..04a3615dde58 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java @@ -27,7 +27,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Uninterruptibles; @@ -65,7 +64,6 @@ /** Tests for {@link BeamFnDataReadRunner}. */ @RunWith(JUnit4.class) public class BeamFnDataReadRunnerTest { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final BeamFnApi.RemoteGrpcPort PORT_SPEC = BeamFnApi.RemoteGrpcPort.newBuilder() .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.getDefaultInstance()).build(); private static final BeamFnApi.FunctionSpec FUNCTION_SPEC = BeamFnApi.FunctionSpec.newBuilder() diff --git a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java index 50fee7a98082..9e50cd0e6212 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java @@ -28,7 +28,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.protobuf.Any; import java.io.IOException; import java.util.ArrayList; @@ -53,7 +52,6 @@ /** Tests for {@link BeamFnDataWriteRunner}. */ @RunWith(JUnit4.class) public class BeamFnDataWriteRunnerTest { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final BeamFnApi.RemoteGrpcPort PORT_SPEC = BeamFnApi.RemoteGrpcPort.newBuilder() .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.getDefaultInstance()).build(); private static final BeamFnApi.FunctionSpec FUNCTION_SPEC = BeamFnApi.FunctionSpec.newBuilder() From f53e5d43d58c79ab9f3d04e112e6f05ad9dfe42f Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Wed, 3 May 2017 17:12:20 -0700 Subject: [PATCH 011/387] [BEAM-2165] Update Flink to support serializing/deserializing custom user types configured via Jackson modules --- runners/flink/pom.xml | 5 ++ .../utils/SerializedPipelineOptions.java | 16 +++- .../runners/flink/PipelineOptionsTest.java | 87 +++++++++++++++++++ 3 files changed, 106 insertions(+), 2 deletions(-) diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index eb2b005ddfd3..41224545cf4f 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -255,6 +255,11 @@ jackson-annotations + + com.fasterxml.jackson.core + jackson-core + + com.fasterxml.jackson.core jackson-databind diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java index 2256bb1df6c8..f717fd76c5e6 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -27,6 +28,7 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Encapsulates the PipelineOptions in serialized form to ship them to the cluster. @@ -42,7 +44,7 @@ public SerializedPipelineOptions(PipelineOptions options) { checkNotNull(options, "PipelineOptions must not be null."); try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - new ObjectMapper().writeValue(baos, options); + createMapper().writeValue(baos, options); this.serializedOptions = baos.toByteArray(); } catch (Exception e) { throw new RuntimeException("Couldn't serialize PipelineOptions.", e); @@ -53,7 +55,7 @@ public SerializedPipelineOptions(PipelineOptions options) { public PipelineOptions getPipelineOptions() { if (pipelineOptions == null) { try { - pipelineOptions = new ObjectMapper().readValue(serializedOptions, PipelineOptions.class); + pipelineOptions = createMapper().readValue(serializedOptions, PipelineOptions.class); IOChannelUtils.registerIOFactoriesAllowOverride(pipelineOptions); FileSystems.setDefaultConfigInWorkers(pipelineOptions); @@ -64,4 +66,14 @@ public PipelineOptions getPipelineOptions() { return pipelineOptions; } + + /** + * Use an {@link ObjectMapper} configured with any {@link Module}s in the class path allowing + * for user specified configuration injection into the ObjectMapper. This supports user custom + * types on {@link PipelineOptions}. + */ + private static ObjectMapper createMapper() { + return new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + } } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java index 23740a135944..7519dbf9d5c1 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java @@ -23,6 +23,23 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.auto.service.AutoService; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.util.Collections; import java.util.HashMap; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; @@ -200,4 +217,74 @@ public void processElement(ProcessContext c) throws Exception { c.getPipelineOptions().as(MyOptions.class).getTestOption()); } } + + /** PipelineOptions used to test auto registration of Jackson modules. */ + public interface JacksonIncompatibleOptions extends PipelineOptions { + JacksonIncompatible getJacksonIncompatible(); + void setJacksonIncompatible(JacksonIncompatible value); + } + + /** A Jackson {@link Module} to test auto-registration of modules. */ + @AutoService(Module.class) + public static class RegisteredTestModule extends SimpleModule { + public RegisteredTestModule() { + super("RegisteredTestModule"); + setMixInAnnotation(JacksonIncompatible.class, JacksonIncompatibleMixin.class); + } + } + + /** A class which Jackson does not know how to serialize/deserialize. */ + public static class JacksonIncompatible { + private final String value; + public JacksonIncompatible(String value) { + this.value = value; + } + } + + /** A Jackson mixin used to add annotations to other classes. */ + @JsonDeserialize(using = JacksonIncompatibleDeserializer.class) + @JsonSerialize(using = JacksonIncompatibleSerializer.class) + public static final class JacksonIncompatibleMixin {} + + /** A Jackson deserializer for {@link JacksonIncompatible}. */ + public static class JacksonIncompatibleDeserializer extends + JsonDeserializer { + + @Override + public JacksonIncompatible deserialize(JsonParser jsonParser, + DeserializationContext deserializationContext) throws IOException, JsonProcessingException { + return new JacksonIncompatible(jsonParser.readValueAs(String.class)); + } + } + + /** A Jackson serializer for {@link JacksonIncompatible}. */ + public static class JacksonIncompatibleSerializer extends JsonSerializer { + + @Override + public void serialize(JacksonIncompatible jacksonIncompatible, JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) throws IOException, JsonProcessingException { + jsonGenerator.writeString(jacksonIncompatible.value); + } + } + + @Test + public void testSerializingPipelineOptionsWithCustomUserType() throws Exception { + String expectedValue = "testValue"; + PipelineOptions options = PipelineOptionsFactory + .fromArgs("--jacksonIncompatible=\"" + expectedValue + "\"") + .as(JacksonIncompatibleOptions.class); + SerializedPipelineOptions context = new SerializedPipelineOptions(options); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + try (ObjectOutputStream outputStream = new ObjectOutputStream(baos)) { + outputStream.writeObject(context); + } + try (ObjectInputStream inputStream = + new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray()))) { + SerializedPipelineOptions copy = (SerializedPipelineOptions) inputStream.readObject(); + assertEquals(expectedValue, + copy.getPipelineOptions().as(JacksonIncompatibleOptions.class) + .getJacksonIncompatible().value); + } + } } From cc654f02e8670ea789aee67508c569e7547ef11f Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Wed, 3 May 2017 13:48:07 -0700 Subject: [PATCH 012/387] [BEAM-1871] Migrate ReleaseInfo away from Google API client GenericJson --- .../beam/runners/dataflow/DataflowRunner.java | 2 +- .../org/apache/beam/sdk/util/ReleaseInfo.java | 70 +++++++++++-------- 2 files changed, 40 insertions(+), 32 deletions(-) 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 9e5a2fb114d6..2b54ba74e7d3 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 @@ -542,7 +542,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { "Unable to submit a job to the Dataflow service with unset version ${pom.version}"); System.out.println("Dataflow SDK version: " + version); - newJob.getEnvironment().setUserAgent(releaseInfo); + newJob.getEnvironment().setUserAgent((Map) releaseInfo.getProperties()); // The Dataflow Service may write to the temporary directory directly, so // must be verified. if (!isNullOrEmpty(options.getGcpTempLocation())) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java index eeac5571c316..c7e24d26d315 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java @@ -17,27 +17,24 @@ */ package org.apache.beam.sdk.util; -import com.google.api.client.json.GenericJson; -import com.google.api.client.util.Key; +import com.google.auto.value.AutoValue; +import com.google.common.collect.ImmutableMap; import java.io.IOException; import java.io.InputStream; +import java.io.Serializable; +import java.util.Map; import java.util.Properties; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * Utilities for working with release information. + * Properties pertaining to this release of Apache Beam. + * + *

Properties will always include a name and version. */ -public final class ReleaseInfo extends GenericJson { - private static final Logger LOG = LoggerFactory.getLogger(ReleaseInfo.class); - - private static final String PROPERTIES_PATH = - "/org/apache/beam/sdk/sdk.properties"; - - private static class LazyInit { - private static final ReleaseInfo INSTANCE = - new ReleaseInfo(PROPERTIES_PATH); - } +@AutoValue +public abstract class ReleaseInfo implements Serializable { + private static final String PROPERTIES_PATH = "/org/apache/beam/sdk/sdk.properties"; /** * Returns an instance of {@link ReleaseInfo}. @@ -46,35 +43,46 @@ public static ReleaseInfo getReleaseInfo() { return LazyInit.INSTANCE; } - @Key private String name = "Apache Beam SDK for Java"; - @Key private String version = "Unknown"; + /** + * Returns an immutable map of all properties pertaining to this release. + */ + public abstract Map getProperties(); /** Provides the SDK name. */ public String getName() { - return name; + return getProperties().get("name"); } /** Provides the SDK version. */ public String getVersion() { - return version; + return getProperties().get("version"); } - private ReleaseInfo(String resourcePath) { - Properties properties = new Properties(); + ///////////////////////////////////////////////////////////////////////// + private static final Logger LOG = LoggerFactory.getLogger(ReleaseInfo.class); + private static final String DEFAULT_NAME = "Apache Beam SDK for Java"; + private static final String DEFAULT_VERSION = "Unknown"; - try (InputStream in = ReleaseInfo.class.getResourceAsStream(PROPERTIES_PATH)) { - if (in == null) { - LOG.warn("Beam properties resource not found: {}", resourcePath); - return; + private static class LazyInit { + private static final ReleaseInfo INSTANCE; + static { + Properties properties = new Properties(); + try (InputStream in = ReleaseInfo.class.getResourceAsStream(PROPERTIES_PATH)) { + if (in == null) { + LOG.warn("Beam properties resource not found: {}", PROPERTIES_PATH); + } else { + properties.load(in); + } + } catch (IOException e) { + LOG.warn("Error loading Beam properties resource: ", e); } - - properties.load(in); - } catch (IOException e) { - LOG.warn("Error loading Beam properties resource: ", e); - } - - for (String name : properties.stringPropertyNames()) { - put(name, properties.getProperty(name)); + if (!properties.containsKey("name")) { + properties.setProperty("name", DEFAULT_NAME); + } + if (!properties.containsKey("version")) { + properties.setProperty("version", DEFAULT_VERSION); + } + INSTANCE = new AutoValue_ReleaseInfo(ImmutableMap.copyOf((Map) properties)); } } } From 8a2dcdb6f9d4839c864a2c46c4b5254d0c7d4760 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 3 May 2017 11:52:02 -0700 Subject: [PATCH 013/387] DataflowRunner: integration test GCP-IO Triggered under `-DskipITs=false -Pdataflow-runner` --- runners/google-cloud-dataflow-java/pom.xml | 50 ++++++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 30ef84d9a461..b0ae757b51aa 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -79,6 +79,49 @@ + + + + dataflow-runner + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + integration-test + + integration-test + verify + + + false + true + all + 4 + + org.apache.beam:beam-sdks-java-io-google-cloud-platform + + + ${integrationTestPipelineOptions} + + + + + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + + @@ -216,6 +259,13 @@ beam-sdks-java-io-google-cloud-platform + + org.apache.beam + beam-sdks-java-io-google-cloud-platform + tests + test + + org.apache.beam beam-runners-core-construction-java From 1671708340fb9fc57cdc91c3bbacdff3ae6af4af Mon Sep 17 00:00:00 2001 From: "yangping.wu" Date: Thu, 4 May 2017 14:04:08 +0800 Subject: [PATCH 014/387] [BEAM-1491]Identify HADOOP_CONF_DIR(or YARN_CONF_DIR) environment variables --- sdks/java/io/hadoop-file-system/pom.xml | 6 + .../sdk/io/hdfs/HadoopFileSystemOptions.java | 73 +++++++++- .../io/hdfs/HadoopFileSystemOptionsTest.java | 125 ++++++++++++++++++ 3 files changed, 197 insertions(+), 7 deletions(-) diff --git a/sdks/java/io/hadoop-file-system/pom.xml b/sdks/java/io/hadoop-file-system/pom.xml index 3ec98482cf74..562277eeb116 100644 --- a/sdks/java/io/hadoop-file-system/pom.xml +++ b/sdks/java/io/hadoop-file-system/pom.xml @@ -186,6 +186,12 @@ test + + org.mockito + mockito-all + test + + junit junit diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptions.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptions.java index 31250bcd5868..45f43e23c80d 100644 --- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptions.java +++ b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptions.java @@ -17,12 +17,22 @@ */ package org.apache.beam.sdk.io.hdfs; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.io.File; import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * {@link PipelineOptions} which encapsulate {@link Configuration Hadoop Configuration} @@ -30,20 +40,69 @@ */ public interface HadoopFileSystemOptions extends PipelineOptions { @Description("A list of Hadoop configurations used to configure zero or more Hadoop filesystems. " - + "To specify on the command-line, represent the value as a JSON list of JSON maps, where " - + "each map represents the entire configuration for a single Hadoop filesystem. For example " - + "--hdfsConfiguration='[{\"fs.default.name\": \"hdfs://localhost:9998\", ...}," + + "By default, Hadoop configuration is loaded from 'core-site.xml' and 'hdfs-site.xml' " + + "based upon the HADOOP_CONF_DIR and YARN_CONF_DIR environment variables. " + + "To specify configuration on the command-line, represent the value as a JSON list of JSON " + + "maps, where each map represents the entire configuration for a single Hadoop filesystem. " + + "For example --hdfsConfiguration='[{\"fs.default.name\": \"hdfs://localhost:9998\", ...}," + "{\"fs.default.name\": \"s3a://\", ...},...]'") @Default.InstanceFactory(ConfigurationLocator.class) List getHdfsConfiguration(); void setHdfsConfiguration(List value); /** A {@link DefaultValueFactory} which locates a Hadoop {@link Configuration}. */ - class ConfigurationLocator implements DefaultValueFactory { + class ConfigurationLocator implements DefaultValueFactory> { + private static final Logger LOG = LoggerFactory.getLogger(ConfigurationLocator.class); @Override - public Configuration create(PipelineOptions options) { - // TODO: Find default configuration to use - return null; + public List create(PipelineOptions options) { + // Find default configuration when HADOOP_CONF_DIR or YARN_CONF_DIR is set. + List configurationList = readConfigurationFromHadoopYarnConfigDirs(); + return configurationList.size() > 0 ? configurationList : null; + } + + private List readConfigurationFromHadoopYarnConfigDirs() { + List configurationList = Lists.newArrayList(); + + /* + * If we find a configuration in HADOOP_CONF_DIR and YARN_CONF_DIR, + * we should be returning them both separately. + * + * Also, ensure that we only load one configuration if both + * HADOOP_CONF_DIR and YARN_CONF_DIR point to the same location. + */ + Set confDirs = Sets.newHashSet(); + for (String confDir : Lists.newArrayList("HADOOP_CONF_DIR", "YARN_CONF_DIR")) { + if (getEnvironment().containsKey(confDir)) { + String hadoopConfDir = getEnvironment().get(confDir); + if (!Strings.isNullOrEmpty(hadoopConfDir)) { + confDirs.add(hadoopConfDir); + } + } + } + + for (String confDir : confDirs) { + if (new File(confDir).exists()) { + Configuration conf = new Configuration(false); + boolean confLoaded = false; + for (String confName : Lists.newArrayList("core-site.xml", "hdfs-site.xml")) { + File confFile = new File(confDir, confName); + if (confFile.exists()) { + LOG.debug("Adding {} to hadoop configuration", confFile.getAbsolutePath()); + conf.addResource(new Path(confFile.getAbsolutePath())); + confLoaded = true; + } + } + if (confLoaded) { + configurationList.add(conf); + } + } + } + return configurationList; + } + + @VisibleForTesting + Map getEnvironment() { + return System.getenv(); } } } diff --git a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptionsTest.java b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptionsTest.java index 634528b8f33f..c99fc5b4ca0f 100644 --- a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptionsTest.java +++ b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptionsTest.java @@ -18,13 +18,26 @@ package org.apache.beam.sdk.io.hdfs; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.AbstractMap; +import java.util.List; import java.util.Map; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.hadoop.conf.Configuration; import org.hamcrest.Matchers; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -33,6 +46,8 @@ */ @RunWith(JUnit4.class) public class HadoopFileSystemOptionsTest { + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Test public void testParsingHdfsConfiguration() { HadoopFileSystemOptions options = PipelineOptionsFactory.fromArgs( @@ -45,4 +60,114 @@ public void testParsingHdfsConfiguration() { assertThat(options.getHdfsConfiguration().get(1), Matchers.>contains( new AbstractMap.SimpleEntry("propertyB", "B"))); } + + @Test + public void testDefaultUnsetEnvHdfsConfiguration() { + HadoopFileSystemOptions.ConfigurationLocator projectFactory = + spy(new HadoopFileSystemOptions.ConfigurationLocator()); + when(projectFactory.getEnvironment()).thenReturn(ImmutableMap.of()); + assertNull(projectFactory.create(PipelineOptionsFactory.create())); + } + + @Test + public void testDefaultJustSetHadoopConfDirConfiguration() throws IOException { + Files.write(createPropertyData("A"), + tmpFolder.newFile("core-site.xml"), StandardCharsets.UTF_8); + Files.write(createPropertyData("B"), + tmpFolder.newFile("hdfs-site.xml"), StandardCharsets.UTF_8); + HadoopFileSystemOptions.ConfigurationLocator configurationLocator = + spy(new HadoopFileSystemOptions.ConfigurationLocator()); + Map environment = Maps.newHashMap(); + environment.put("HADOOP_CONF_DIR", tmpFolder.getRoot().getAbsolutePath()); + when(configurationLocator.getEnvironment()).thenReturn(environment); + + List configurationList = + configurationLocator.create(PipelineOptionsFactory.create()); + assertEquals(1, configurationList.size()); + assertThat(configurationList.get(0).get("propertyA"), Matchers.equalTo("A")); + assertThat(configurationList.get(0).get("propertyB"), Matchers.equalTo("B")); + } + + @Test + public void testDefaultJustSetYarnConfDirConfiguration() throws IOException { + Files.write(createPropertyData("A"), + tmpFolder.newFile("core-site.xml"), StandardCharsets.UTF_8); + Files.write(createPropertyData("B"), + tmpFolder.newFile("hdfs-site.xml"), StandardCharsets.UTF_8); + HadoopFileSystemOptions.ConfigurationLocator configurationLocator = + spy(new HadoopFileSystemOptions.ConfigurationLocator()); + Map environment = Maps.newHashMap(); + environment.put("YARN_CONF_DIR", tmpFolder.getRoot().getAbsolutePath()); + when(configurationLocator.getEnvironment()).thenReturn(environment); + + List configurationList = + configurationLocator.create(PipelineOptionsFactory.create()); + assertEquals(1, configurationList.size()); + assertThat(configurationList.get(0).get("propertyA"), Matchers.equalTo("A")); + assertThat(configurationList.get(0).get("propertyB"), Matchers.equalTo("B")); + } + + @Test + public void testDefaultSetYarnConfDirAndHadoopConfDirAndSameConfiguration() throws IOException { + Files.write(createPropertyData("A"), + tmpFolder.newFile("core-site.xml"), StandardCharsets.UTF_8); + Files.write(createPropertyData("B"), + tmpFolder.newFile("hdfs-site.xml"), StandardCharsets.UTF_8); + HadoopFileSystemOptions.ConfigurationLocator configurationLocator = + spy(new HadoopFileSystemOptions.ConfigurationLocator()); + Map environment = Maps.newHashMap(); + environment.put("YARN_CONF_DIR", tmpFolder.getRoot().getAbsolutePath()); + environment.put("HADOOP_CONF_DIR", tmpFolder.getRoot().getAbsolutePath()); + when(configurationLocator.getEnvironment()).thenReturn(environment); + + List configurationList = + configurationLocator.create(PipelineOptionsFactory.create()); + assertEquals(1, configurationList.size()); + assertThat(configurationList.get(0).get("propertyA"), Matchers.equalTo("A")); + assertThat(configurationList.get(0).get("propertyB"), Matchers.equalTo("B")); + } + + @Test + public void testDefaultSetYarnConfDirAndHadoopConfDirNotSameConfiguration() throws IOException { + File hadoopConfDir = tmpFolder.newFolder("hadoop"); + File yarnConfDir = tmpFolder.newFolder("yarn"); + Files.write(createPropertyData("A"), + new File(hadoopConfDir, "core-site.xml"), StandardCharsets.UTF_8); + Files.write(createPropertyData("B"), + new File(hadoopConfDir, "hdfs-site.xml"), StandardCharsets.UTF_8); + Files.write(createPropertyData("C"), + new File(yarnConfDir, "core-site.xml"), StandardCharsets.UTF_8); + Files.write(createPropertyData("D"), + new File(yarnConfDir, "hdfs-site.xml"), StandardCharsets.UTF_8); + HadoopFileSystemOptions.ConfigurationLocator configurationLocator = + spy(new HadoopFileSystemOptions.ConfigurationLocator()); + Map environment = Maps.newHashMap(); + environment.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath()); + environment.put("HADOOP_CONF_DIR", yarnConfDir.getAbsolutePath()); + when(configurationLocator.getEnvironment()).thenReturn(environment); + + List configurationList = + configurationLocator.create(PipelineOptionsFactory.create()); + assertEquals(2, configurationList.size()); + int hadoopConfIndex = configurationList.get(0).get("propertyA") != null ? 0 : 1; + assertThat( + configurationList.get(hadoopConfIndex).get("propertyA"), Matchers.equalTo("A")); + assertThat( + configurationList.get(hadoopConfIndex).get("propertyB"), Matchers.equalTo("B")); + assertThat( + configurationList.get(1 - hadoopConfIndex).get("propertyC"), Matchers.equalTo("C")); + assertThat( + configurationList.get(1 - hadoopConfIndex).get("propertyD"), Matchers.equalTo("D")); + } + + private static String createPropertyData(String property) { + return "\n" + + "\n" + + "\n" + + " \n" + + " property" + property + "\n" + + " " + property + "\n" + + " \n" + + ""; + } } From fba3d87ffec08f84c8be08ee16942b13364da2d9 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 3 May 2017 14:56:37 -0700 Subject: [PATCH 015/387] Split Coder's encode/decode methods into two methods depending on context. This allows the outer context to be marked deprecated. A follow-up PR will remove the old method once all consumers have been updated. --- .../org/apache/beam/sdk/coders/Coder.java | 72 +++++++++++++++++++ .../beam/sdk/coders/StructuredCoder.java | 61 ++++++++++++++++ 2 files changed, 133 insertions(+) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index 8ba8ad3e6c23..c923719eb025 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -59,6 +59,7 @@ */ public interface Coder extends Serializable { /** The context in which encoding or decoding is being done. */ + @Deprecated class Context { /** * The outer context: the value being encoded or decoded takes @@ -110,6 +111,28 @@ public String toString() { } } + /** + * Encodes the given value of type {@code T} onto the given output stream. + * + * @throws IOException if writing to the {@code OutputStream} fails + * for some reason + * @throws CoderException if the value could not be encoded for some reason + */ + void encode(T value, OutputStream outStream) + throws CoderException, IOException; + + /** + * Encodes the given value of type {@code T} onto the given output stream + * in the outer context. + * + * @throws IOException if writing to the {@code OutputStream} fails + * for some reason + * @throws CoderException if the value could not be encoded for some reason + */ + @Deprecated + void encodeOuter(T value, OutputStream outStream) + throws CoderException, IOException; + /** * Encodes the given value of type {@code T} onto the given output stream * in the given context. @@ -118,6 +141,7 @@ public String toString() { * for some reason * @throws CoderException if the value could not be encoded for some reason */ + @Deprecated void encode(T value, OutputStream outStream, Context context) throws CoderException, IOException; @@ -129,6 +153,28 @@ void encode(T value, OutputStream outStream, Context context) * for some reason * @throws CoderException if the value could not be decoded for some reason */ + T decode(InputStream inStream) throws CoderException, IOException; + + /** + * Decodes a value of type {@code T} from the given input stream in + * the outer context. Returns the decoded value. + * + * @throws IOException if reading from the {@code InputStream} fails + * for some reason + * @throws CoderException if the value could not be decoded for some reason + */ + @Deprecated + T decodeOuter(InputStream inStream) throws CoderException, IOException; + + /** + * Decodes a value of type {@code T} from the given input stream in + * the given context. Returns the decoded value. + * + * @throws IOException if reading from the {@code InputStream} fails + * for some reason + * @throws CoderException if the value could not be decoded for some reason + */ + @Deprecated T decode(InputStream inStream, Context context) throws CoderException, IOException; @@ -200,6 +246,19 @@ T decode(InputStream inStream, Context context) * {@link org.apache.beam.sdk.runners.PipelineRunner} * implementations. */ + boolean isRegisterByteSizeObserverCheap(T value); + + /** + * Returns whether {@link #registerByteSizeObserver} cheap enough to + * call for every element, that is, if this {@code Coder} can + * calculate the byte size of the element to be coded in roughly + * constant time (or lazily). + * + *

Not intended to be called by user code, but instead by + * {@link org.apache.beam.sdk.runners.PipelineRunner} + * implementations. + */ + @Deprecated boolean isRegisterByteSizeObserverCheap(T value, Context context); /** @@ -210,6 +269,19 @@ T decode(InputStream inStream, Context context) * {@link org.apache.beam.sdk.runners.PipelineRunner} * implementations. */ + void registerByteSizeObserver( + T value, ElementByteSizeObserver observer) + throws Exception; + + /** + * Notifies the {@code ElementByteSizeObserver} about the byte size + * of the encoded value using this {@code Coder}. + * + *

Not intended to be called by user code, but instead by + * {@link org.apache.beam.sdk.runners.PipelineRunner} + * implementations. + */ + @Deprecated void registerByteSizeObserver( T value, ElementByteSizeObserver observer, Context context) throws Exception; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java index 0cd53b0573c8..cc39429a8e20 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java @@ -20,6 +20,9 @@ import com.google.common.io.ByteStreams; import com.google.common.io.CountingOutputStream; import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -98,6 +101,58 @@ public String toString() { return builder.toString(); } + public void encode(T value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Coder.Context.NESTED); + } + + @Deprecated + public void encodeOuter(T value, OutputStream outStream) + throws CoderException, IOException { + encode(value, outStream, Coder.Context.OUTER); + } + + @Deprecated + public void encode(T value, OutputStream outStream, Coder.Context context) + throws CoderException, IOException { + if (context == Coder.Context.NESTED) { + encode(value, outStream); + } else { + encodeOuter(value, outStream); + } + } + + public T decode(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Coder.Context.NESTED); + } + + @Deprecated + public T decodeOuter(InputStream inStream) throws CoderException, IOException { + return decode(inStream, Coder.Context.OUTER); + } + + @Deprecated + public T decode(InputStream inStream, Coder.Context context) + throws CoderException, IOException { + if (context == Coder.Context.NESTED) { + return decode(inStream); + } else { + return decodeOuter(inStream); + } + } + + /** + * {@inheritDoc} + * + * @return {@code false} unless it is overridden. {@link StructuredCoder#registerByteSizeObserver} + * invokes {@link #getEncodedElementByteSize} which requires re-encoding an element + * unless it is overridden. This is considered expensive. + */ + @Override + public boolean isRegisterByteSizeObserverCheap(T value) { + return isRegisterByteSizeObserverCheap(value, Context.NESTED); + } + /** * {@inheritDoc} * @@ -124,6 +179,12 @@ protected long getEncodedElementByteSize(T value, Context context) } } + @Override + public void registerByteSizeObserver(T value, ElementByteSizeObserver observer) + throws Exception { + registerByteSizeObserver(value, observer, Context.NESTED); + } + /** * {@inheritDoc} * From 690ec3b1f7b6ce9caaa7b9e401878e136f44bc50 Mon Sep 17 00:00:00 2001 From: bchambers Date: Wed, 3 May 2017 16:40:09 -0700 Subject: [PATCH 016/387] [BEAM-2162] Add logging to long BigQuery jobs --- .../io/gcp/bigquery/BigQueryServicesImpl.java | 20 ++++++++++++++++++- .../io/gcp/bigquery/BigQuerySourceBase.java | 2 ++ 2 files changed, 21 insertions(+), 1 deletion(-) 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 8e395f0da7f1..b348abdd22df 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.RetryHttpRequestInitializer; import org.apache.beam.sdk.util.Transport; import org.joda.time.Duration; +import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,6 +81,9 @@ class BigQueryServicesImpl implements BigQueryServices { private static final Logger LOG = LoggerFactory.getLogger(BigQueryServicesImpl.class); + // How frequently to log while polling. + private static final Duration POLLING_LOG_GAP = Duration.standardMinutes(10); + // The maximum number of retries to execute a BigQuery RPC. private static final int MAX_RPC_RETRIES = 9; @@ -219,7 +223,8 @@ static void startJob( do { try { client.jobs().insert(jobRef.getProjectId(), job).execute(); - LOG.info("Started BigQuery job: {}.", jobRef); + LOG.info("Started BigQuery job: {}.\n{}", jobRef, + formatBqStatusCommand(jobRef.getProjectId(), jobRef.getJobId())); return; // SUCCEEDED } catch (GoogleJsonResponseException e) { if (errorExtractor.itemAlreadyExists(e)) { @@ -257,6 +262,7 @@ Job pollJob( JobReference jobRef, Sleeper sleeper, BackOff backoff) throws InterruptedException { + Instant nextLog = Instant.now().plus(POLLING_LOG_GAP); do { try { Job job = client.jobs().get(jobRef.getProjectId(), jobRef.getJobId()).execute(); @@ -265,6 +271,13 @@ Job pollJob( return job; } // The job is not DONE, wait longer and retry. + if (Instant.now().isAfter(nextLog)) { + LOG.info("Still waiting for BigQuery job {}\n{}", + jobRef.getJobId(), + formatBqStatusCommand( + jobRef.getProjectId(), jobRef.getJobId())); + nextLog = Instant.now().plus(POLLING_LOG_GAP); + } } catch (IOException e) { // ignore and retry LOG.info("Ignore the error and retry polling job status.", e); @@ -274,6 +287,11 @@ Job pollJob( return null; } + private static String formatBqStatusCommand(String projectId, String jobId) { + return String.format("bq show -j --format=prettyjson --project_id=%s %s", + projectId, jobId); + } + @Override public JobStatistics dryRunQuery(String projectId, JobConfigurationQuery queryConfig) throws InterruptedException, IOException { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java index 41e298c56fb9..49000d610924 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java @@ -141,6 +141,8 @@ private List executeExtract( BigQueryHelpers.statusToPrettyString(extractJob.getStatus()))); } + LOG.info("BigQuery extract job completed: {}", jobId); + List tempFiles = BigQueryIO.getExtractFilePaths(extractDestinationDir, extractJob); return ImmutableList.copyOf(tempFiles); } From 17ad1efe7355b238efb5e341487a8e22660b3b77 Mon Sep 17 00:00:00 2001 From: Borisa Zivkovic Date: Wed, 3 May 2017 16:22:18 +0100 Subject: [PATCH 017/387] Use BinaryCombineLongFn in GroupIntoBatches --- .../beam/sdk/transforms/GroupIntoBatches.java | 51 +++++++------------ 1 file changed, 17 insertions(+), 34 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 02f3a8598f88..564baf4e8b3b 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 @@ -23,7 +23,6 @@ import com.google.common.collect.Iterables; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timer; @@ -115,7 +114,7 @@ static class GroupIntoBatchesDoFn private final StateSpec> batchSpec; @StateId(NUM_ELEMENTS_IN_BATCH_ID) - private final StateSpec> + private final StateSpec> numElementsInBatchSpec; @StateId(KEY_ID) @@ -131,35 +130,19 @@ static class GroupIntoBatchesDoFn this.batchSize = batchSize; this.allowedLateness = allowedLateness; this.batchSpec = StateSpecs.bag(inputValueCoder); - this.numElementsInBatchSpec = - StateSpecs.combining( - VarLongCoder.of(), - new Combine.CombineFn() { - - @Override - public Long createAccumulator() { - return 0L; - } - - @Override - public Long addInput(Long accumulator, Long input) { - return accumulator + input; - } - - @Override - public Long mergeAccumulators(Iterable accumulators) { - long sum = 0L; - for (Long accumulator : accumulators) { - sum += accumulator; - } - return sum; - } - - @Override - public Long extractOutput(Long accumulator) { - return accumulator; - } - }); + this.numElementsInBatchSpec = StateSpecs.combining(new Combine.BinaryCombineLongFn() { + + @Override + public long identity() { + return 0L; + } + + @Override + public long apply(long left, long right) { + return left + right; + } + + }); this.keySpec = StateSpecs.value(inputKeyCoder); // prefetch every 20% of batchSize elements. Do not prefetch if batchSize is too little @@ -171,7 +154,7 @@ public void processElement( @TimerId(END_OF_WINDOW_ID) Timer timer, @StateId(BATCH_ID) BagState batch, @StateId(NUM_ELEMENTS_IN_BATCH_ID) - CombiningState numElementsInBatch, + CombiningState numElementsInBatch, @StateId(KEY_ID) ValueState key, ProcessContext c, BoundedWindow window) { @@ -203,7 +186,7 @@ public void onTimerCallback( @StateId(KEY_ID) ValueState key, @StateId(BATCH_ID) BagState batch, @StateId(NUM_ELEMENTS_IN_BATCH_ID) - CombiningState numElementsInBatch, + CombiningState numElementsInBatch, BoundedWindow window) { LOGGER.debug( "*** END OF WINDOW *** for timer timestamp {} in windows {}", @@ -215,7 +198,7 @@ private void flushBatch( Context c, ValueState key, BagState batch, - CombiningState numElementsInBatch) { + CombiningState numElementsInBatch) { Iterable values = batch.read(); // when the timer fires, batch state might be empty if (!Iterables.isEmpty(values)) { From 70dad36f099ea0b454e2900302f7e7f866579f79 Mon Sep 17 00:00:00 2001 From: Sourabh Bajaj Date: Wed, 3 May 2017 13:50:46 -0700 Subject: [PATCH 018/387] [BEAM-2152] Remove gcloud auth as application default credentials does it --- sdks/python/apache_beam/internal/gcp/auth.py | 32 ------------------- .../runners/dataflow/internal/apiclient.py | 3 +- 2 files changed, 1 insertion(+), 34 deletions(-) diff --git a/sdks/python/apache_beam/internal/gcp/auth.py b/sdks/python/apache_beam/internal/gcp/auth.py index 53c2d077c8aa..3c8dd64d9759 100644 --- a/sdks/python/apache_beam/internal/gcp/auth.py +++ b/sdks/python/apache_beam/internal/gcp/auth.py @@ -26,7 +26,6 @@ from oauth2client.client import GoogleCredentials from oauth2client.client import OAuth2Credentials -from apache_beam.utils import processes from apache_beam.utils import retry @@ -94,27 +93,6 @@ def _refresh(self, http_request): datetime.timedelta(seconds=token_data['expires_in'])) -class _GCloudWrapperCredentials(OAuth2Credentials): - """Credentials class wrapping gcloud credentials via shell.""" - - def __init__(self, user_agent, **kwds): - super(_GCloudWrapperCredentials, self).__init__( - None, None, None, None, None, None, user_agent, **kwds) - - def _refresh(self, http_request): - """Gets an access token using the gcloud client.""" - try: - gcloud_process = processes.Popen( - ['gcloud', 'auth', 'print-access-token'], stdout=processes.PIPE) - except OSError: - message = 'gcloud tool not found so falling back to using ' +\ - 'application default credentials' - logging.warning(message) - raise AuthenticationException(message) - output, _ = gcloud_process.communicate() - self.access_token = output.strip() - - def get_service_credentials(): """Get credentials to access Google services.""" user_agent = 'beam-python-sdk/1.0' @@ -134,16 +112,6 @@ def get_service_credentials(): 'https://www.googleapis.com/auth/datastore' ] - try: - credentials = _GCloudWrapperCredentials(user_agent) - # Check if we are able to get an access token. If not fallback to - # application default credentials. - credentials.get_access_token() - return credentials - except AuthenticationException: - logging.warning('Unable to find credentials from gcloud.') - - # Falling back to application default credentials. try: credentials = GoogleCredentials.get_application_default() credentials = credentials.create_scoped(client_scopes) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index 26ed7b694d9a..e64cf8da3a13 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -411,8 +411,7 @@ def stage_file(self, gcs_or_local_path, file_name, stream, if e.status_code in reportable_errors: raise IOError(('Could not upload to GCS path %s: %s. Please verify ' 'that credentials are valid and that you have write ' - 'access to the specified path. Stale credentials can be ' - 'refreshed by executing "gcloud auth login".') % + 'access to the specified path.') % (gcs_or_local_path, reportable_errors[e.status_code])) raise logging.info('Completed GCS upload to %s', gcs_location) From c102d277e22cef8001c0f78d3a5ed00000e8d99d Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 3 May 2017 17:50:20 -0700 Subject: [PATCH 019/387] AvroIOTest: stop using IOChannelUtils, remove invalid test --- .../org/apache/beam/sdk/io/AvroIOTest.java | 28 ------------------- 1 file changed, 28 deletions(-) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java index 1506aa9cbc6d..47b847fc153e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java @@ -57,11 +57,9 @@ import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.testing.UsesTestStream; import org.apache.beam.sdk.testing.ValidatesRunner; @@ -71,14 +69,11 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; import org.joda.time.Instant; -import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -98,11 +93,6 @@ public class AvroIOTest { @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); - @BeforeClass - public static void setupClass() { - IOChannelUtils.registerIOFactoriesAllowOverride(TestPipeline.testingPipelineOptions()); - } - @Test public void testAvroIOGetName() { assertEquals("AvroIO.Read", AvroIO.read(String.class).from("/tmp/foo*/baz").getName()); @@ -570,22 +560,4 @@ public void testWriteDisplayData() { assertThat(displayData, hasDisplayItem("numShards", 100)); assertThat(displayData, hasDisplayItem("codec", CodecFactory.snappyCodec().toString())); } - - @Test - @Category(ValidatesRunner.class) - @Ignore("[BEAM-436] DirectRunner ValidatesRunner tempLocation configuration insufficient") - public void testPrimitiveWriteDisplayData() throws IOException { - PipelineOptions options = DisplayDataEvaluator.getDefaultOptions(); - String tempRoot = options.as(TestPipelineOptions.class).getTempRoot(); - String outputPath = IOChannelUtils.getFactory(tempRoot).resolve(tempRoot, "foo"); - - DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(options); - - AvroIO.Write write = - AvroIO.writeGenericRecords(Schema.create(Schema.Type.STRING)).to(outputPath); - - Set displayData = evaluator.displayDataForPrimitiveTransforms(write); - assertThat("AvroIO.Write should include the file pattern in its primitive transform", - displayData, hasItem(hasDisplayItem("fileNamePattern"))); - } } From e5a38ed2610b8ef72192e5a1b9a5630578300164 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 3 May 2017 17:55:32 -0700 Subject: [PATCH 020/387] DataflowRunner: switch from IOChannels to FileSystems for creating files --- .../apache/beam/runners/dataflow/DataflowRunner.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) 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 2b54ba74e7d3..1a806b9faecb 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 @@ -84,6 +84,7 @@ import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.FileBasedSink; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.WriteFiles; @@ -114,7 +115,6 @@ import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.InstanceBuilder; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.NameUtils; @@ -192,9 +192,6 @@ public class DataflowRunner extends PipelineRunner { * @return The newly created runner. */ public static DataflowRunner fromOptions(PipelineOptions options) { - // (Re-)register standard IO factories. Clobbers any prior credentials. - IOChannelUtils.registerIOFactoriesAllowOverride(options); - DataflowPipelineOptions dataflowOptions = PipelineOptionsValidator.validate(DataflowPipelineOptions.class, options); ArrayList missing = new ArrayList<>(); @@ -578,9 +575,11 @@ public DataflowPipelineJob run(Pipeline pipeline) { fileLocation.startsWith("/") || fileLocation.startsWith("gs://"), "Location must be local or on Cloud Storage, got %s.", fileLocation); + ResourceId fileResource = FileSystems.matchNewResource(fileLocation, false /* isDirectory */); String workSpecJson = DataflowPipelineTranslator.jobToString(newJob); - try (PrintWriter printWriter = new PrintWriter( - Channels.newOutputStream(IOChannelUtils.create(fileLocation, MimeTypes.TEXT)))) { + try (PrintWriter printWriter = + new PrintWriter( + Channels.newOutputStream(FileSystems.create(fileResource, MimeTypes.TEXT)))) { printWriter.print(workSpecJson); LOG.info("Printed job specification to {}", fileLocation); } catch (IOException ex) { From dc9e00485afaf737557bc6a82750e45ecba34926 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 3 May 2017 18:01:29 -0700 Subject: [PATCH 021/387] DataflowPipelineOptions: use FileSystems, not IOChannelUtils, to resolve staging location --- .../options/DataflowPipelineOptions.java | 14 +++++--------- .../DataflowPipelineTranslatorTest.java | 2 +- .../runners/dataflow/DataflowRunnerTest.java | 4 +++- .../options/DataflowPipelineOptionsTest.java | 19 +++++++++---------- 4 files changed, 18 insertions(+), 21 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java index 0796b6d5dd16..11618af9b1fe 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java @@ -17,11 +17,12 @@ */ package org.apache.beam.runners.dataflow.options; -import java.io.IOException; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryOptions; import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; import org.apache.beam.sdk.options.ApplicationNameOptions; @@ -32,7 +33,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.options.Validation; -import org.apache.beam.sdk.util.IOChannelUtils; /** * Options that can be used to configure the {@link DataflowRunner}. @@ -137,13 +137,9 @@ public String create(PipelineOptions options) { "Error constructing default value for stagingLocation: gcpTempLocation is not" + " a valid GCS path, %s. ", gcpTempLocation), e); } - try { - return IOChannelUtils.resolve(gcpTempLocation, "staging"); - } catch (IOException e) { - throw new IllegalArgumentException(String.format( - "Unable to resolve stagingLocation from gcpTempLocation: %s." - + " Please set the staging location explicitly.", gcpTempLocation), e); - } + return FileSystems.matchNewResource(gcpTempLocation, true /* isDirectory */) + .resolve("staging", StandardResolveOptions.RESOLVE_DIRECTORY) + .toString(); } } } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index a6ad8c5ca2fd..70f00fb0f2a0 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -224,7 +224,7 @@ public void testSettingOfSdkPipelineOptions() throws IOException { assertThat(optionsMap, hasEntry("jobName", (Object) "some-job-name")); assertThat(optionsMap, hasEntry("tempLocation", (Object) "gs://somebucket/some/path")); assertThat(optionsMap, - hasEntry("stagingLocation", (Object) "gs://somebucket/some/path/staging")); + hasEntry("stagingLocation", (Object) "gs://somebucket/some/path/staging/")); assertThat(optionsMap, hasEntry("stableUniqueNames", (Object) "WARNING")); assertThat(optionsMap, hasEntry("streaming", (Object) false)); assertThat(optionsMap, hasEntry("numberOfWorkerHarnessThreads", (Object) 0)); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index c0dfbee6951e..5aebf29d1033 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -157,8 +157,9 @@ public List answer(InvocationOnMock invocation) throws Throwable { } }); when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(VALID_STAGING_BUCKET))).thenReturn(true); + when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(VALID_STAGING_BUCKET))).thenReturn(true); when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(VALID_TEMP_BUCKET))).thenReturn(true); - when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(VALID_TEMP_BUCKET + "/staging"))). + when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(VALID_TEMP_BUCKET + "/staging/"))). thenReturn(true); when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(VALID_PROFILE_BUCKET))).thenReturn(true); when(mockGcsUtil.bucketAccessible(GcsPath.fromUri(NON_EXISTENT_BUCKET))).thenReturn(false); @@ -826,6 +827,7 @@ public void testProjectDescription() throws IOException { @Test public void testInvalidNumberOfWorkerHarnessThreads() throws IOException { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + FileSystems.setDefaultConfigInWorkers(options); options.setRunner(DataflowRunner.class); options.setProject("foo-12345"); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java index 30eee0e9a8b0..8b8fd6c83c96 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java @@ -22,10 +22,10 @@ import static org.junit.Assert.assertTrue; import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.ResetDateTimeProvider; import org.apache.beam.sdk.testing.RestoreSystemProperties; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.NoopPathValidator; import org.junit.Rule; import org.junit.Test; @@ -127,7 +127,6 @@ public void testUtf8UserNameAndApplicationNameIsNormalized() { @Test public void testStagingLocation() { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); - IOChannelUtils.registerIOFactoriesAllowOverride(options); options.setPathValidatorClass(NoopPathValidator.class); options.setTempLocation("gs://temp_location"); options.setStagingLocation("gs://staging_location"); @@ -138,21 +137,21 @@ public void testStagingLocation() { @Test public void testDefaultToTempLocation() { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); - IOChannelUtils.registerIOFactoriesAllowOverride(options); + FileSystems.setDefaultConfigInWorkers(options); options.setPathValidatorClass(NoopPathValidator.class); - options.setTempLocation("gs://temp_location"); - assertEquals("gs://temp_location", options.getGcpTempLocation()); - assertEquals("gs://temp_location/staging", options.getStagingLocation()); + options.setTempLocation("gs://temp_location/"); + assertEquals("gs://temp_location/", options.getGcpTempLocation()); + assertEquals("gs://temp_location/staging/", options.getStagingLocation()); } @Test public void testDefaultToGcpTempLocation() { DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); - IOChannelUtils.registerIOFactoriesAllowOverride(options); + FileSystems.setDefaultConfigInWorkers(options); options.setPathValidatorClass(NoopPathValidator.class); - options.setTempLocation("gs://temp_location"); - options.setGcpTempLocation("gs://gcp_temp_location"); - assertEquals("gs://gcp_temp_location/staging", options.getStagingLocation()); + options.setTempLocation("gs://temp_location/"); + options.setGcpTempLocation("gs://gcp_temp_location/"); + assertEquals("gs://gcp_temp_location/staging/", options.getStagingLocation()); } @Test From 5519fe51aea3e95fa0ea4a36c7b036917b7f94ec Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 3 May 2017 18:02:42 -0700 Subject: [PATCH 022/387] Stop registering IOChannelFactories in SDK harness, runners, tests --- .../apex/translation/utils/SerializablePipelineOptions.java | 2 -- .../flink/translation/utils/SerializedPipelineOptions.java | 2 -- .../beam/runners/spark/translation/SparkRuntimeContext.java | 4 +--- .../main/java/org/apache/beam/sdk/runners/PipelineRunner.java | 4 +--- .../main/java/org/apache/beam/sdk/testing/TestPipeline.java | 2 -- .../src/main/java/org/apache/beam/fn/harness/FnHarness.java | 3 --- 6 files changed, 2 insertions(+), 15 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java index 14476b57c26c..02afa7a31e46 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java @@ -27,7 +27,6 @@ import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.common.ReflectHelpers; /** @@ -63,7 +62,6 @@ public void readExternal(ObjectInput in) throws IOException, ClassNotFoundExcept .as(ApexPipelineOptions.class); if (FILE_SYSTEMS_INTIIALIZED.compareAndSet(false, true)) { - IOChannelUtils.registerIOFactoriesAllowOverride(pipelineOptions); FileSystems.setDefaultConfigInWorkers(pipelineOptions); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java index f717fd76c5e6..84f3bf40f59c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java @@ -27,7 +27,6 @@ import java.io.Serializable; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.common.ReflectHelpers; /** @@ -57,7 +56,6 @@ public PipelineOptions getPipelineOptions() { try { pipelineOptions = createMapper().readValue(serializedOptions, PipelineOptions.class); - IOChannelUtils.registerIOFactoriesAllowOverride(pipelineOptions); FileSystems.setDefaultConfigInWorkers(pipelineOptions); } catch (IOException e) { throw new RuntimeException("Couldn't deserialize the PipelineOptions.", e); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java index 6bba863ae657..e006143cc0f3 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java @@ -26,7 +26,6 @@ import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.IOChannelUtils; /** * The SparkRuntimeContext allows us to define useful features on the client side before our @@ -78,8 +77,7 @@ static PipelineOptions getOrInit(String serializedPipelineOptions) { pipelineOptions = deserializePipelineOptions(serializedPipelineOptions); } } - // register IO factories. - IOChannelUtils.registerIOFactoriesAllowOverride(pipelineOptions); + // Register standard FileSystems. FileSystems.setDefaultConfigInWorkers(pipelineOptions); } return pipelineOptions; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java index a318dfca99bf..229e04ffa12d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.InstanceBuilder; /** @@ -44,8 +43,7 @@ public static PipelineRunner fromOptions(PipelineOptio checkNotNull(options); PipelineOptionsValidator.validate(PipelineOptions.class, options); - // (Re-)register standard IO factories. Clobbers any prior credentials. - IOChannelUtils.registerIOFactoriesAllowOverride(options); + // (Re-)register standard FileSystems. Clobbers any prior credentials. FileSystems.setDefaultConfigInWorkers(options); @SuppressWarnings("unchecked") diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index 4d0cc2b36423..868dcbd56d55 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -46,7 +46,6 @@ import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.common.ReflectHelpers; import org.junit.experimental.categories.Category; import org.junit.rules.TestRule; @@ -400,7 +399,6 @@ public static PipelineOptions testingPipelineOptions() { } options.setStableUniqueNames(CheckEnabled.ERROR); - IOChannelUtils.registerIOFactoriesAllowOverride(options); FileSystems.setDefaultConfigInWorkers(options); return options; } catch (IOException e) { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index 24f826c5c522..05ab44f7ff5a 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -33,7 +33,6 @@ import org.apache.beam.fn.v1.BeamFnApi; import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.common.ReflectHelpers; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,8 +89,6 @@ public static void main(String[] args) throws Exception { public static void main(PipelineOptions options, BeamFnApi.ApiServiceDescriptor loggingApiServiceDescriptor, BeamFnApi.ApiServiceDescriptor controlApiServiceDescriptor) throws Exception { - IOChannelUtils.registerIOFactories(options); - ManagedChannelFactory channelFactory = ManagedChannelFactory.from(options); StreamObserverFactory streamObserverFactory = StreamObserverFactory.fromOptions(options); PrintStream originalErrStream = System.err; From f4e7c02d0a91f5df9ddcb2e868ef5adb86b3f85c Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 3 May 2017 19:34:04 -0700 Subject: [PATCH 023/387] FileSystems: add match(String) for single spec --- .../org/apache/beam/sdk/io/FileSystems.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) 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 0110a0cc269e..a3af8d9860e9 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 @@ -19,6 +19,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Verify.verify; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; @@ -107,6 +108,22 @@ public static List match(List specs) throws IOException { return getFileSystemInternal(getOnlyScheme(specs)).match(specs); } + + /** + * Like {@link #match(List)}, but for a single resource specification. + * + *

The function {@link #match(List)} is preferred when matching multiple patterns, as it allows + * for bulk API calls to remote filesystems. + */ + public static MatchResult match(String spec) throws IOException { + List matches = match(Collections.singletonList(spec)); + verify( + matches.size() == 1, + "FileSystem implementation for %s did not return exactly one MatchResult: %s", + spec, + matches); + return matches.get(0); + } /** * Returns the {@link Metadata} for a single file resource. Expects a resource specification * {@code spec} that matches a single result. From d050477065cf8cf27b1334ed7bebad5b30e5d8e4 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 3 May 2017 18:11:34 -0700 Subject: [PATCH 024/387] WriteWithShardingFactoryTest: switch to FileSystems --- .../direct/WriteWithShardingFactoryTest.java | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java index 18940d2d5bcc..f28c8cf3aede 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java @@ -32,7 +32,6 @@ import java.io.Reader; import java.nio.CharBuffer; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.UUID; @@ -42,9 +41,11 @@ import org.apache.beam.sdk.io.DefaultFilenamePolicy; 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.LocalResources; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.WriteFiles; +import org.apache.beam.sdk.io.fs.MatchResult.Metadata; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.TestPipeline; @@ -53,7 +54,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; @@ -72,7 +72,7 @@ */ @RunWith(JUnit4.class) public class WriteWithShardingFactoryTest { - public static final int INPUT_SIZE = 10000; + private static final int INPUT_SIZE = 10000; @Rule public TemporaryFolder tmp = new TemporaryFolder(); private WriteWithShardingFactory factory = new WriteWithShardingFactory<>(); @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false); @@ -86,19 +86,22 @@ public void dynamicallyReshardedWrite() throws Exception { Collections.shuffle(strs); String fileName = "resharded_write"; - String outputPath = tmp.getRoot().getAbsolutePath(); - String targetLocation = IOChannelUtils.resolve(outputPath, fileName); + String targetLocation = tmp.getRoot().toPath().resolve(fileName).toString(); + String targetLocationGlob = targetLocation + '*'; + // TextIO is implemented in terms of the WriteFiles PTransform. When sharding is not specified, // resharding should be automatically applied p.apply(Create.of(strs)).apply(TextIO.write().to(targetLocation)); - p.run(); - Collection files = IOChannelUtils.getFactory(outputPath).match(targetLocation + "*"); + List matches = FileSystems.match(targetLocationGlob).metadata(); List actuals = new ArrayList<>(strs.size()); - for (String file : files) { - CharBuffer buf = CharBuffer.allocate((int) new File(file).length()); - try (Reader reader = new FileReader(file)) { + List files = new ArrayList<>(strs.size()); + for (Metadata match : matches) { + String filename = match.resourceId().toString(); + files.add(filename); + CharBuffer buf = CharBuffer.allocate((int) new File(filename).length()); + try (Reader reader = new FileReader(filename)) { reader.read(buf); buf.flip(); } From 56b1280d51b2458de93c7a5e2f31822fe6cf1718 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 3 May 2017 18:18:52 -0700 Subject: [PATCH 025/387] ReadTranslator: remove outdated comment --- .../java/org/apache/beam/runners/dataflow/ReadTranslator.java | 2 -- 1 file changed, 2 deletions(-) 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 c304c3260642..396c3051e5f8 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 @@ -46,8 +46,6 @@ public static void translateReadHelper(Source source, PTransform transform, TranslationContext context) { try { - // TODO: Move this validation out of translation once IOChannelUtils is portable - // and can be reconstructed on the worker. if (source instanceof FileBasedSource) { ValueProvider filePatternOrSpec = ((FileBasedSource) source).getFileOrPatternSpecProvider(); From 7e3d5723db6d1118df31e6af496b1aa2240607c9 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 3 May 2017 19:34:25 -0700 Subject: [PATCH 026/387] TFRecordIO: switch to FileSystems from IOChannelUtils --- .../src/main/java/org/apache/beam/sdk/io/TFRecordIO.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java index 3198829c00ef..6350fb57aa98 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.Read.Bounded; +import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; @@ -43,7 +44,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -158,12 +158,11 @@ public PCollection expand(PBegin input) { if (getValidate()) { checkState(getFilepattern().isAccessible(), "Cannot validate with a RVP."); try { + MatchResult matches = FileSystems.match(getFilepattern().get()); checkState( - !IOChannelUtils.getFactory(getFilepattern().get()) - .match(getFilepattern().get()) - .isEmpty(), + !matches.metadata().isEmpty(), "Unable to find any files matching %s", - getFilepattern()); + getFilepattern().get()); } catch (IOException e) { throw new IllegalStateException( String.format("Failed to validate %s", getFilepattern().get()), e); From ab8f92fcc272e7eca70720fb91df584769376ecc Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 2 May 2017 17:53:57 -0700 Subject: [PATCH 027/387] Remove POutput#recordAsOutput Add PValue#setDefaultName, for composite POutputs to use when they are being finalized. Expand POutput#finishSpecifyingOutput to take the name of the transform that is being finished. --- .../beam/sdk/runners/TransformHierarchy.java | 8 ++-- .../apache/beam/sdk/values/PCollection.java | 4 +- .../beam/sdk/values/PCollectionList.java | 14 +++--- .../beam/sdk/values/PCollectionTuple.java | 17 ++++--- .../org/apache/beam/sdk/values/PDone.java | 12 ++--- .../org/apache/beam/sdk/values/POutput.java | 25 ++--------- .../apache/beam/sdk/values/PValueBase.java | 44 ++++++------------- .../sdk/runners/TransformHierarchyTest.java | 3 +- .../apache/beam/sdk/transforms/ParDoTest.java | 4 +- .../beam/sdk/io/gcp/bigquery/WriteResult.java | 21 +-------- 10 files changed, 47 insertions(+), 105 deletions(-) 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 92361942fc8a..703aeb684346 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 @@ -171,14 +171,14 @@ public void setOutput(POutput output) { for (PValue value : output.expand().values()) { if (!producers.containsKey(value)) { producers.put(value, current); + value.finishSpecifyingOutput( + current.getFullName(), unexpandedInputs.get(current), current.transform); } - value.finishSpecifyingOutput(unexpandedInputs.get(current), current.transform); producerInput.put(value, unexpandedInputs.get(current)); } - output.finishSpecifyingOutput(unexpandedInputs.get(current), current.transform); + output.finishSpecifyingOutput( + current.getFullName(), unexpandedInputs.get(current), current.transform); current.setOutput(output); - // TODO: Replace with a "generateDefaultNames" method. - output.recordAsOutput(current.toAppliedPTransform()); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java index 20e5d68946b4..16223223749c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java @@ -83,9 +83,9 @@ public class PCollection extends PValueBase implements PValue { @Override public void finishSpecifyingOutput( - PInput input, PTransform transform) { + String transformName, PInput input, PTransform transform) { this.coderOrFailure = inferCoderOrFail(input, transform, getPipeline().getCoderRegistry()); - super.finishSpecifyingOutput(input, transform); + super.finishSpecifyingOutput(transformName, input, transform); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java index 7b45deb391a3..48c36495ef0d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionList.java @@ -24,7 +24,6 @@ import java.util.Map; import java.util.Objects; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Partition; @@ -230,21 +229,20 @@ public Map, PValue> expand() { } @Override - public void recordAsOutput(AppliedPTransform transform) { + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) { + // All component PCollections will have already been finished. int i = 0; for (TaggedPValue tpv : pcollections) { @SuppressWarnings("unchecked") PCollection pc = (PCollection) tpv.getValue(); - pc.recordAsOutput(transform, "out" + i); + if (pc.getName().equals(PValueBase.defaultName(transformName))) { + pc.setName(String.format("%s.%s%s", transformName, "out", i)); + } i++; } } - @Override - public void finishSpecifyingOutput(PInput input, PTransform transform) { - // All component PCollections will have already been finished. - } - @Override public boolean equals(Object other) { if (!(other instanceof PCollectionList)) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java index ce67e94764bc..5027df652ddf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java @@ -23,7 +23,6 @@ import java.util.Map; import java.util.Objects; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.WindowingStrategy; @@ -236,22 +235,22 @@ public Map, PValue> expand() { } @Override - public void recordAsOutput(AppliedPTransform transform) { + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) { + // All component PCollections will already have been finished. Update their names if + // appropriate. int i = 0; for (Map.Entry, PCollection> entry - : pcollectionMap.entrySet()) { + : pcollectionMap.entrySet()) { TupleTag tag = entry.getKey(); PCollection pc = entry.getValue(); - pc.recordAsOutput(transform, tag.getOutName(i)); + if (pc.getName().equals(PValueBase.defaultName(transformName))) { + pc.setName(String.format("%s.%s", transformName, tag.getOutName(i))); + } i++; } } - @Override - public void finishSpecifyingOutput(PInput input, PTransform transform) { - // All component PCollections will already have been finished - } - @Override public boolean equals(Object other) { if (!(other instanceof PCollectionTuple)) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java index 5c9800de9d1d..92473b94e6a4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java @@ -21,7 +21,6 @@ import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.WriteFiles; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; /** @@ -56,13 +55,8 @@ public Map, PValue> expand() { return Collections.emptyMap(); } - /** Does nothing; there are no concrete outputs to record. */ + /** Does nothing; there is nothing to finish specifying. */ @Override - public void recordAsOutput(AppliedPTransform transform) {} - - /** - * Does nothing; there is nothing to finish specifying. - */ - @Override - public void finishSpecifyingOutput(PInput input, PTransform transform) { } + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) {} } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java index bb01beb2d0c6..c6d15e327f0f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutput.java @@ -19,7 +19,6 @@ import java.util.Map; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; /** @@ -47,19 +46,6 @@ public interface POutput { */ Map, PValue> expand(); - /** - * Records that this {@code POutput} is an output of the given - * {@code PTransform}. - * - *

For a compound {@code POutput}, it is advised to call - * this method on each component {@code POutput}. - * - *

This is not intended to be invoked by user code, but - * is automatically invoked as part of applying the - * producing {@link PTransform}. - */ - void recordAsOutput(AppliedPTransform transform); - /** * As part of applying the producing {@link PTransform}, finalizes this output to make it ready * for being used as an input and for running. @@ -67,12 +53,9 @@ public interface POutput { *

This includes ensuring that all {@link PCollection PCollections} have {@link * org.apache.beam.sdk.coders.Coder Coders} specified or defaulted. * - *

Automatically invoked whenever this {@link POutput} is output, after {@link - * PValue#finishSpecifyingOutput(PInput, PTransform)} has been called on each component {@link - * PValue} returned by {@link #expand()}. - * - * @deprecated see BEAM-1199 + *

Automatically invoked whenever this {@link POutput} is output, after + * {@link PValue#finishSpecifyingOutput(String, PInput, PTransform)} has been called on each + * component {@link PValue} returned by {@link #expand()}. */ - @Deprecated - void finishSpecifyingOutput(PInput input, PTransform transform); + void finishSpecifyingOutput(String transformName, PInput input, PTransform transform); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java index 7ab58088fb90..4de0589d1222 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java @@ -17,10 +17,11 @@ */ package org.apache.beam.sdk.values; +import static com.google.common.base.Preconditions.checkState; + import java.util.Collections; import java.util.Map; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.NameUtils; @@ -65,10 +66,7 @@ public String getName() { * already been finalized and may no longer be set. */ public PValueBase setName(String name) { - if (finishedSpecifying) { - throw new IllegalStateException( - "cannot change the name of " + this + " once it's been used"); - } + checkState(!finishedSpecifying, "cannot change the name of %s once it's been used", this); this.name = name; return this; } @@ -104,26 +102,6 @@ protected PValueBase() { */ private boolean finishedSpecifying = false; - @Override - public void recordAsOutput(AppliedPTransform transform) { - recordAsOutput(transform, "out"); - } - - /** - * Records that this {@link PValueBase} is an output with the - * given name of the given {@link AppliedPTransform} in the given - * {@link Pipeline}. - * - *

To be invoked only by {@link POutput#recordAsOutput} - * implementations. Not to be invoked directly by user code. - */ - protected void recordAsOutput(AppliedPTransform transform, - String outName) { - if (name == null) { - name = transform.getFullName() + "." + outName; - } - } - /** * Returns whether this {@link PValueBase} has been finalized, and * its core properties, e.g., name, can no longer be changed. @@ -165,11 +143,15 @@ public Pipeline getPipeline() { return pipeline; } - /** - * Default behavior for {@link #finishSpecifyingOutput(PInput, PTransform)}} is - * to do nothing. Override if your {@link PValue} requires - * finalization. - */ @Override - public void finishSpecifyingOutput(PInput input, PTransform transform) { } + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) { + if (name == null) { + setName(defaultName(transformName)); + } + } + + static String defaultName(String transformName) { + return String.format("%s.%s", transformName, "out"); + } } 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 1b884e2dd365..e495758c788e 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 @@ -145,7 +145,8 @@ public void producingOwnAndOthersOutputsFails() { final PCollectionList appended = pcList.and( PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED)); + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED) + .setName("prim")); hierarchy.pushNode( "AddPc", pcList, 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 26904aa09d15..d4475c925266 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 @@ -1244,7 +1244,9 @@ public void testTaggedOutputUnregisteredExplicitCoder() throws Exception { outputTuple.get(additionalOutputTag).apply(View.asSingleton()); assertEquals(new TestDummyCoder(), outputTuple.get(additionalOutputTag).getCoder()); - outputTuple.get(additionalOutputTag).finishSpecifyingOutput(input, pardo); // Check for crashes + outputTuple + .get(additionalOutputTag) + .finishSpecifyingOutput("ParDo", input, pardo); // Check for crashes assertEquals(new TestDummyCoder(), outputTuple.get(additionalOutputTag).getCoder()); // Check for corruption } 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 d137f052f554..bc18e8e032e9 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 @@ -20,7 +20,6 @@ import java.util.Collections; import java.util.Map; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; @@ -55,23 +54,7 @@ public Pipeline getPipeline() { return pipeline; } - /** - * Records that this {@link WriteResult} is an output with the given name of the given {@link - * AppliedPTransform}. - * - *

By default, does nothing. - * - *

To be invoked only by {@link POutput#recordAsOutput} implementations. Not to be invoked - * directly by user code. - */ - @Override - public void recordAsOutput(AppliedPTransform transform) {} - - /** - * Default behavior for {@link #finishSpecifyingOutput(PInput, PTransform)}} is - * to do nothing. Override if your {@link PValue} requires - * finalization. - */ @Override - public void finishSpecifyingOutput(PInput input, PTransform transform) { } + public void finishSpecifyingOutput( + String transformName, PInput input, PTransform transform) {} } From c141ebc4ac8088c782d6b74ee38042a72ed342f3 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 3 May 2017 17:39:32 -0700 Subject: [PATCH 028/387] One more TaggedOutput name change. --- sdks/python/apache_beam/runners/common.pxd | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/common.pxd b/sdks/python/apache_beam/runners/common.pxd index f3395c1ddf10..53f7aa8cc962 100644 --- a/sdks/python/apache_beam/runners/common.pxd +++ b/sdks/python/apache_beam/runners/common.pxd @@ -21,7 +21,7 @@ from apache_beam.utils.windowed_value cimport WindowedValue from apache_beam.metrics.execution cimport ScopedMetricsContainer -cdef type OutputValue, TimestampedValue +cdef type TaggedOutput, TimestampedValue cdef class Receiver(object): From 4cdfe36ce3b423dfb0f26a371a255d5490e7b593 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Tue, 2 May 2017 12:01:38 -0700 Subject: [PATCH 029/387] Update JavaDoc of checkpoint, and expose `enable.auto.commit` for more visible. --- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) 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 8f94b8a488b0..8ab33d1c5b69 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 @@ -204,14 +204,23 @@ * *

Partition Assignment and Checkpointing

* The Kafka partitions are evenly distributed among splits (workers). - * Checkpointing is fully supported and each split can resume from previous checkpoint. See - * {@link UnboundedKafkaSource#split(int, PipelineOptions)} for more details on + * + *

Checkpointing is fully supported and each split can resume from previous checkpoint + * (to the extent supported by runner). + * See {@link UnboundedKafkaSource#split(int, PipelineOptions)} for more details on * splits and checkpoint support. * - *

When the pipeline starts for the first time without any checkpoint, the source starts + *

When the pipeline starts for the first time, or without any checkpoint, the source starts * consuming from the latest offsets. You can override this behavior to consume from the * beginning by setting appropriate appropriate properties in {@link ConsumerConfig}, through * {@link Read#updateConsumerProperties(Map)}. + * You can also enable offset auto_commit in Kafka to resume from last committed. + * + *

In summary, KafkaIO.read follows below sequence to set initial offset:
+ * 1. {@link KafkaCheckpointMark} provided by runner;
+ * 2. Consumer offset stored in Kafka when + * {@code ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG = true};
+ * 3. Start from latest offset by default; * *

Writing to Kafka

* From 48c8ed17623b3f36b7aeebb2e2ca585a259d2fec Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 4 May 2017 09:08:28 -0700 Subject: [PATCH 030/387] Use CloudObject encodings in the Beam Fn Harness This mostly reverts commit e2b5d6ea8a5d41ac27245a4999f59a73dfe24c43. --- sdks/java/harness/pom.xml | 10 ------ .../runners/core/BeamFnDataReadRunner.java | 21 +++++++++--- .../runners/core/BeamFnDataWriteRunner.java | 22 ++++++++++--- .../control/ProcessBundleHandlerTest.java | 32 +++++++++---------- .../core/BeamFnDataReadRunnerTest.java | 16 ++++++---- .../core/BeamFnDataWriteRunnerTest.java | 16 ++++++---- 6 files changed, 68 insertions(+), 49 deletions(-) diff --git a/sdks/java/harness/pom.xml b/sdks/java/harness/pom.xml index 73f08cc012fb..d00dfe4b14ce 100644 --- a/sdks/java/harness/pom.xml +++ b/sdks/java/harness/pom.xml @@ -63,16 +63,6 @@ beam-sdks-java-core - - org.apache.beam - beam-runners-core-construction-java - - - - org.apache.beam - beam-sdks-common-runner-api - - org.apache.beam beam-sdks-java-core diff --git a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java index 7c4a5e8fa272..e6928d1aa43b 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java @@ -18,8 +18,10 @@ package org.apache.beam.runners.core; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; +import com.google.protobuf.BytesValue; import java.io.IOException; import java.util.Collection; import java.util.Map; @@ -28,9 +30,9 @@ import org.apache.beam.fn.harness.data.BeamFnDataClient; import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.v1.BeamFnApi; -import org.apache.beam.runners.core.construction.Coders; +import org.apache.beam.runners.dataflow.util.CloudObject; +import org.apache.beam.runners.dataflow.util.CloudObjects; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.common.runner.v1.RunnerApi.MessageWithComponents; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.slf4j.Logger; @@ -47,6 +49,8 @@ public class BeamFnDataReadRunner { private static final Logger LOG = LoggerFactory.getLogger(BeamFnDataReadRunner.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private final BeamFnApi.ApiServiceDescriptor apiServiceDescriptor; private final Collection>> consumers; private final Supplier processBundleInstructionIdSupplier; @@ -71,12 +75,19 @@ public BeamFnDataReadRunner( this.beamFnDataClientFactory = beamFnDataClientFactory; this.consumers = ImmutableList.copyOf(FluentIterable.concat(outputMap.values())); - MessageWithComponents runnerApiCoder = - coderSpec.getFunctionSpec().getData().unpack(MessageWithComponents.class); @SuppressWarnings("unchecked") Coder> coder = (Coder>) - Coders.fromProto(runnerApiCoder.getCoder(), runnerApiCoder.getComponents()); + CloudObjects.coderFromCloudObject( + CloudObject.fromSpec( + OBJECT_MAPPER.readValue( + coderSpec + .getFunctionSpec() + .getData() + .unpack(BytesValue.class) + .getValue() + .newInput(), + Map.class))); this.coder = coder; } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java index 3a11def393fc..a78da5d201d7 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java @@ -18,14 +18,17 @@ package org.apache.beam.runners.core; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.protobuf.BytesValue; import java.io.IOException; +import java.util.Map; import java.util.function.Supplier; import org.apache.beam.fn.harness.data.BeamFnDataClient; import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer; import org.apache.beam.fn.v1.BeamFnApi; -import org.apache.beam.runners.core.construction.Coders; +import org.apache.beam.runners.dataflow.util.CloudObject; +import org.apache.beam.runners.dataflow.util.CloudObjects; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.common.runner.v1.RunnerApi.MessageWithComponents; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; @@ -37,6 +40,8 @@ * For each request, call {@link #registerForOutput()} to start and call {@link #close()} to finish. */ public class BeamFnDataWriteRunner { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private final BeamFnApi.ApiServiceDescriptor apiServiceDescriptor; private final BeamFnApi.Target outputTarget; private final Coder> coder; @@ -58,12 +63,19 @@ public BeamFnDataWriteRunner( this.processBundleInstructionIdSupplier = processBundleInstructionIdSupplier; this.outputTarget = outputTarget; - MessageWithComponents runnerApiCoder = - coderSpec.getFunctionSpec().getData().unpack(MessageWithComponents.class); @SuppressWarnings("unchecked") Coder> coder = (Coder>) - Coders.fromProto(runnerApiCoder.getCoder(), runnerApiCoder.getComponents()); + CloudObjects.coderFromCloudObject( + CloudObject.fromSpec( + OBJECT_MAPPER.readValue( + coderSpec + .getFunctionSpec() + .getData() + .unpack(BytesValue.class) + .getValue() + .newInput(), + Map.class))); this.coder = coder; } 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 654f98988cc0..1cdd087cfd47 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 @@ -33,6 +33,7 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; @@ -61,7 +62,7 @@ import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; import org.apache.beam.fn.v1.BeamFnApi; -import org.apache.beam.runners.core.construction.Coders; +import org.apache.beam.runners.dataflow.util.CloudObjects; import org.apache.beam.runners.dataflow.util.DoFnInfo; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -91,6 +92,8 @@ /** Tests for {@link ProcessBundleHandler}. */ @RunWith(JUnit4.class) public class ProcessBundleHandlerTest { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Coder> STRING_CODER = WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); private static final String LONG_CODER_SPEC_ID = "998L"; @@ -105,23 +108,18 @@ public class ProcessBundleHandlerTest { static { try { STRING_CODER_SPEC = - BeamFnApi.Coder.newBuilder() - .setFunctionSpec( - BeamFnApi.FunctionSpec.newBuilder() - .setId(STRING_CODER_SPEC_ID) - .setData(Any.pack(Coders.toProto(STRING_CODER)))) - .build(); + BeamFnApi.Coder.newBuilder().setFunctionSpec(BeamFnApi.FunctionSpec.newBuilder() + .setId(STRING_CODER_SPEC_ID) + .setData(Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom( + OBJECT_MAPPER.writeValueAsBytes(CloudObjects.asCloudObject(STRING_CODER)))).build()))) + .build(); LONG_CODER_SPEC = - BeamFnApi.Coder.newBuilder() - .setFunctionSpec( - BeamFnApi.FunctionSpec.newBuilder() - .setId(STRING_CODER_SPEC_ID) - .setData( - Any.pack( - Coders.toProto( - WindowedValue.getFullCoder( - VarLongCoder.of(), GlobalWindow.Coder.INSTANCE))))) - .build(); + BeamFnApi.Coder.newBuilder().setFunctionSpec(BeamFnApi.FunctionSpec.newBuilder() + .setId(STRING_CODER_SPEC_ID) + .setData(Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom( + OBJECT_MAPPER.writeValueAsBytes(CloudObjects.asCloudObject(WindowedValue.getFullCoder( + VarLongCoder.of(), GlobalWindow.Coder.INSTANCE))))).build()))) + .build(); } catch (IOException e) { throw new ExceptionInInitializerError(e); } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java index 04a3615dde58..a3d4a1b0f109 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java @@ -27,10 +27,13 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Uninterruptibles; import com.google.protobuf.Any; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -45,7 +48,7 @@ import org.apache.beam.fn.harness.test.TestExecutors; import org.apache.beam.fn.harness.test.TestExecutors.TestExecutorService; import org.apache.beam.fn.v1.BeamFnApi; -import org.apache.beam.runners.core.construction.Coders; +import org.apache.beam.runners.dataflow.util.CloudObjects; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -64,6 +67,8 @@ /** Tests for {@link BeamFnDataReadRunner}. */ @RunWith(JUnit4.class) public class BeamFnDataReadRunnerTest { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final BeamFnApi.RemoteGrpcPort PORT_SPEC = BeamFnApi.RemoteGrpcPort.newBuilder() .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.getDefaultInstance()).build(); private static final BeamFnApi.FunctionSpec FUNCTION_SPEC = BeamFnApi.FunctionSpec.newBuilder() @@ -73,11 +78,10 @@ public class BeamFnDataReadRunnerTest { private static final BeamFnApi.Coder CODER_SPEC; static { try { - CODER_SPEC = - BeamFnApi.Coder.newBuilder() - .setFunctionSpec( - BeamFnApi.FunctionSpec.newBuilder().setData(Any.pack(Coders.toProto(CODER)))) - .build(); + CODER_SPEC = BeamFnApi.Coder.newBuilder().setFunctionSpec(BeamFnApi.FunctionSpec.newBuilder() + .setData(Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom( + OBJECT_MAPPER.writeValueAsBytes(CloudObjects.asCloudObject(CODER)))).build()))) + .build(); } catch (IOException e) { throw new ExceptionInInitializerError(e); } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java index 9e50cd0e6212..338396650b41 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java @@ -28,14 +28,17 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.protobuf.Any; +import com.google.protobuf.ByteString; +import com.google.protobuf.BytesValue; import java.io.IOException; import java.util.ArrayList; import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.fn.harness.data.BeamFnDataClient; import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer; import org.apache.beam.fn.v1.BeamFnApi; -import org.apache.beam.runners.core.construction.Coders; +import org.apache.beam.runners.dataflow.util.CloudObjects; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -52,6 +55,8 @@ /** Tests for {@link BeamFnDataWriteRunner}. */ @RunWith(JUnit4.class) public class BeamFnDataWriteRunnerTest { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final BeamFnApi.RemoteGrpcPort PORT_SPEC = BeamFnApi.RemoteGrpcPort.newBuilder() .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.getDefaultInstance()).build(); private static final BeamFnApi.FunctionSpec FUNCTION_SPEC = BeamFnApi.FunctionSpec.newBuilder() @@ -61,11 +66,10 @@ public class BeamFnDataWriteRunnerTest { private static final BeamFnApi.Coder CODER_SPEC; static { try { - CODER_SPEC = - BeamFnApi.Coder.newBuilder() - .setFunctionSpec( - BeamFnApi.FunctionSpec.newBuilder().setData(Any.pack(Coders.toProto(CODER)))) - .build(); + CODER_SPEC = BeamFnApi.Coder.newBuilder().setFunctionSpec(BeamFnApi.FunctionSpec.newBuilder() + .setData(Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom( + OBJECT_MAPPER.writeValueAsBytes(CloudObjects.asCloudObject(CODER)))).build()))) + .build(); } catch (IOException e) { throw new ExceptionInInitializerError(e); } From 95ade45eced4787eb67a9d4d13dae48ffb176919 Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Tue, 2 May 2017 19:00:29 +0300 Subject: [PATCH 031/387] [BEAM-1763] Verify PAssert execution in runners which support metrics. --- .../beam/runners/flink/FlinkRunner.java | 3 ++ .../beam/runners/spark/TestSparkRunner.java | 47 ------------------- .../ResumeFromCheckpointStreamingTest.java | 12 +++-- .../beam/sdk/metrics/MetricsEnvironment.java | 5 ++ .../apache/beam/sdk/testing/TestPipeline.java | 46 ++++++++++++++++-- 5 files changed, 57 insertions(+), 56 deletions(-) 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 181ffda7571c..a5972efa1a15 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 @@ -31,6 +31,7 @@ import java.util.TreeSet; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PipelineRunner; @@ -103,6 +104,8 @@ private FlinkRunner(FlinkPipelineOptions options) { public PipelineResult run(Pipeline pipeline) { logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline); + MetricsEnvironment.setMetricsSupported(true); + LOG.info("Executing pipeline using FlinkRunner."); FlinkPipelineExecutionEnvironment env = new FlinkPipelineExecutionEnvironment(options); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index 10e98b825803..1e678131d92b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -40,15 +40,11 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.BoundedReadFromUnboundedSource; -import org.apache.beam.sdk.metrics.MetricNameFilter; -import org.apache.beam.sdk.metrics.MetricResult; -import org.apache.beam.sdk.metrics.MetricsFilter; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; @@ -116,8 +112,6 @@ public SparkPipelineResult run(Pipeline pipeline) { } SparkPipelineResult result = null; - int expectedNumberOfAssertions = PAssert.countAsserts(pipeline); - // clear state of Aggregators, Metrics and Watermarks if exists. AggregatorsAccumulator.clear(); MetricsAccumulator.clear(); @@ -137,47 +131,6 @@ public SparkPipelineResult run(Pipeline pipeline) { String.format("Finish state %s is not allowed.", finishState), finishState, isOneOf(PipelineResult.State.STOPPED, PipelineResult.State.DONE)); - - // validate assertion succeeded (at least once). - long successAssertions = 0; - Iterable> counterResults = result.metrics().queryMetrics( - MetricsFilter.builder() - .addNameFilter(MetricNameFilter.named(PAssert.class, PAssert.SUCCESS_COUNTER)) - .build()).counters(); - for (MetricResult counter : counterResults) { - if (counter.attempted().longValue() > 0) { - successAssertions++; - } - } - Integer expectedAssertions = testSparkPipelineOptions.getExpectedAssertions() != null - ? testSparkPipelineOptions.getExpectedAssertions() : expectedNumberOfAssertions; - assertThat( - String.format( - "Expected %d successful assertions, but found %d.", - expectedAssertions, successAssertions), - successAssertions, - is(expectedAssertions.longValue())); - // validate assertion didn't fail. - long failedAssertions = 0; - Iterable> failCounterResults = result.metrics().queryMetrics( - MetricsFilter.builder() - .addNameFilter(MetricNameFilter.named(PAssert.class, PAssert.FAILURE_COUNTER)) - .build()).counters(); - for (MetricResult counter : failCounterResults) { - if (counter.attempted().longValue() > 0) { - failedAssertions++; - } - } - assertThat( - String.format("Found %d failed assertions.", failedAssertions), - failedAssertions, - is(0L)); - - LOG.info( - String.format( - "Successfully asserted pipeline %s with %d successful assertions.", - testSparkPipelineOptions.getJobName(), - successAssertions)); } finally { try { // cleanup checkpoint dir. diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java index 7d7fd0880382..33571f0f1691 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java @@ -180,7 +180,8 @@ public void testWithResume() throws Exception { long successAssertions = 0; Iterable> counterResults = res.metrics().queryMetrics( MetricsFilter.builder() - .addNameFilter(MetricNameFilter.named(PAssert.class, PAssert.SUCCESS_COUNTER)) + .addNameFilter( + MetricNameFilter.named(PAssertWithoutFlatten.class, PAssert.SUCCESS_COUNTER)) .build()).counters(); for (MetricResult counter : counterResults) { if (counter.attempted().longValue() > 0) { @@ -196,7 +197,8 @@ public void testWithResume() throws Exception { long failedAssertions = 0; Iterable> failCounterResults = res.metrics().queryMetrics( MetricsFilter.builder() - .addNameFilter(MetricNameFilter.named(PAssert.class, PAssert.FAILURE_COUNTER)) + .addNameFilter(MetricNameFilter.named( + PAssertWithoutFlatten.class, PAssert.FAILURE_COUNTER)) .build()).counters(); for (MetricResult counter : failCounterResults) { if (counter.attempted().longValue() > 0) { @@ -330,8 +332,10 @@ public PDone expand(PCollection> input) { } private static class AssertDoFn extends DoFn, Void> { - private final Counter success = Metrics.counter(PAssert.class, PAssert.SUCCESS_COUNTER); - private final Counter failure = Metrics.counter(PAssert.class, PAssert.FAILURE_COUNTER); + private final Counter success = + Metrics.counter(PAssertWithoutFlatten.class, PAssert.SUCCESS_COUNTER); + private final Counter failure = + Metrics.counter(PAssertWithoutFlatten.class, PAssert.FAILURE_COUNTER); private final T[] expected; AssertDoFn(T[] expected) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java index 2942578b1e95..a4b311ffd43e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsEnvironment.java @@ -70,6 +70,11 @@ public static void setMetricsSupported(boolean supported) { METRICS_SUPPORTED.set(supported); } + /** Indicates whether metrics reporting is supported. */ + public static boolean isMetricsSupported() { + return METRICS_SUPPORTED.get(); + } + /** * Set the {@link MetricsContainer} for the current thread. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index 868dcbd56d55..d8fe51d92c72 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.testing; import static com.google.common.base.Preconditions.checkState; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.TreeNode; @@ -41,6 +43,10 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.metrics.MetricNameFilter; +import org.apache.beam.sdk.metrics.MetricResult; +import org.apache.beam.sdk.metrics.MetricsEnvironment; +import org.apache.beam.sdk.metrics.MetricsFilter; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled; @@ -186,8 +192,8 @@ private void verifyPipelineExecution() { if (pipelineRunSucceeded() && !visitedAll(pipelineNodes)) { final boolean hasDanglingPAssert = FluentIterable.from(pipelineNodes) - .filter(Predicates.not(Predicates.in(runVisitedNodes))) - .anyMatch(isPAssertNode); + .filter(Predicates.not(Predicates.in(runVisitedNodes))) + .anyMatch(isPAssertNode); if (hasDanglingPAssert) { throw new AbandonedNodeException("The pipeline contains abandoned PAssert(s)."); } else { @@ -319,12 +325,13 @@ public PipelineResult run() { checkState( enforcement.isPresent(), "Is your TestPipeline declaration missing a @Rule annotation? Usage: " - + "@Rule public final transient TestPipeline pipeline = TestPipeline.create();"); + + "@Rule public final transient TestPipeline pipeline = TestPipeline.create();"); final PipelineResult pipelineResult; try { enforcement.get().beforePipelineExecution(); pipelineResult = super.run(); + verifyPAssertsSucceeded(pipelineResult); } catch (RuntimeException exc) { Throwable cause = exc.getCause(); if (cause instanceof AssertionError) { @@ -385,8 +392,8 @@ public static PipelineOptions testingPipelineOptions() { Strings.isNullOrEmpty(beamTestPipelineOptions) ? PipelineOptionsFactory.create() : PipelineOptionsFactory.fromArgs( - MAPPER.readValue(beamTestPipelineOptions, String[].class)) - .as(TestPipelineOptions.class); + MAPPER.readValue(beamTestPipelineOptions, String[].class)) + .as(TestPipelineOptions.class); options.as(ApplicationNameOptions.class).setAppName(getAppName()); // If no options were specified, set some reasonable defaults @@ -488,6 +495,35 @@ private static Optional findCallersStackTrace() { return firstInstanceAfterTestPipeline; } + /** + * Verifies all {{@link PAssert PAsserts}} in the pipeline have been executed and were successful. + * + *

Note this only runs for runners which support Metrics. Runners which do not should verify + * this in some other way. See: https://issues.apache.org/jira/browse/BEAM-2001

+ */ + private void verifyPAssertsSucceeded(PipelineResult pipelineResult) { + if (MetricsEnvironment.isMetricsSupported()) { + long expectedNumberOfAssertions = (long) PAssert.countAsserts(this); + + long successfulAssertions = 0; + Iterable> successCounterResults = + pipelineResult.metrics().queryMetrics( + MetricsFilter.builder() + .addNameFilter(MetricNameFilter.named(PAssert.class, PAssert.SUCCESS_COUNTER)) + .build()) + .counters(); + for (MetricResult counter : successCounterResults) { + if (counter.attempted() > 0) { + successfulAssertions++; + } + } + + assertThat(String + .format("Expected %d successful assertions, but found %d.", expectedNumberOfAssertions, + successfulAssertions), successfulAssertions, is(expectedNumberOfAssertions)); + } + } + private static class IsEmptyVisitor extends PipelineVisitor.Defaults { private boolean empty = true; From 5555040d935c67f5cd48f2ffe2721a07fe6e0a50 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Sat, 18 Mar 2017 12:16:06 +0100 Subject: [PATCH 032/387] [BEAM-1726] Fix empty side inputs in Flink Streaming Runner --- .../beam/runners/core/SideInputHandler.java | 10 +++---- .../wrappers/streaming/DoFnOperator.java | 27 ++++++++++++++++++- 2 files changed, 31 insertions(+), 6 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java index 5c67148ea0e6..b29f9d0281bb 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -161,11 +162,6 @@ public void addSideInputValue( @Override public T get(PCollectionView sideInput, BoundedWindow window) { - if (!isReady(sideInput, window)) { - throw new IllegalStateException( - "Side input " + sideInput + " is not ready for window " + window); - } - @SuppressWarnings("unchecked") Coder windowCoder = (Coder) sideInput @@ -181,6 +177,10 @@ public T get(PCollectionView sideInput, BoundedWindow window) { Iterable> elements = state.read(); + if (elements == null) { + elements = Collections.emptyList(); + } + return sideInput.getViewFn().apply(elements); } 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 c62403637fc6..16bf5d253419 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 @@ -463,7 +463,32 @@ public void processWatermark1(Watermark mark) throws Exception { @Override public void processWatermark2(Watermark mark) throws Exception { - // ignore watermarks from the side-input input + if (mark.getTimestamp() == BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { + // this means we will never see any more side input + pushbackDoFnRunner.startBundle(); + + BagState> pushedBack = + pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag); + + 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()); + + pushbackDoFnRunner.finishBundle(); + + // maybe output a new watermark + processWatermark1(new Watermark(currentInputWatermark)); + } } @Override From 7c44935e1c47cce2ecfe842e37c2cf89f48d8583 Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Sat, 18 Mar 2017 15:21:45 +0200 Subject: [PATCH 033/387] [BEAM-1726] Fix RuntimeException throwing in FlinkStateInternals --- .../wrappers/streaming/state/FlinkStateInternals.java | 2 ++ 1 file changed, 2 insertions(+) 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 c033be672d9a..cea6e0f719fb 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 @@ -592,6 +592,8 @@ public void add(InputT value) { } current = combineFn.addInput(current, value); state.update(current); + } catch (RuntimeException re) { + throw re; } catch (Exception e) { throw new RuntimeException("Error adding to state." , e); } From 0e2bb1808350cbebf771d0971deb06787732800d Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Sun, 19 Mar 2017 07:49:08 +0100 Subject: [PATCH 034/387] [BEAM-1726] Fix Flatten with input copies in Flink Streaming Runner --- .../FlinkStreamingTransformTranslators.java | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) 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 c024493f2885..7339c01d9b81 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 @@ -966,10 +966,36 @@ public void flatMap( } else { DataStream result = null; + + // Determine DataStreams that we use as input several times. For those, we need to uniquify + // input streams because Flink seems to swallow watermarks when we have a union of one and + // the same stream. + Map, Integer> duplicates = new HashMap<>(); + for (PValue input : allInputs.values()) { + DataStream current = context.getInputDataStream(input); + Integer oldValue = duplicates.put(current, 1); + if (oldValue != null) { + duplicates.put(current, oldValue + 1); + } + } + for (PValue input : allInputs.values()) { DataStream current = context.getInputDataStream(input); + + final Integer timesRequired = duplicates.get(current); + if (timesRequired > 1) { + current = current.flatMap(new FlatMapFunction() { + private static final long serialVersionUID = 1L; + + @Override + public void flatMap(T t, Collector collector) throws Exception { + collector.collect(t); + } + }); + } result = (result == null) ? current : result.union(current); } + context.setOutputDataStream(context.getOutput(transform), result); } } From 8d91a97b77fbda74c577d2cdbd507395834e147c Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Wed, 3 May 2017 21:06:00 +0300 Subject: [PATCH 035/387] [BEAM-1763] Replace usage of PipelineRule with TestPipeline in Spark runner tests --- runners/spark/pom.xml | 47 +++++++- .../runners/spark/SparkRunnerRegistrar.java | 4 +- .../apache/beam/runners/spark/CacheTest.java | 12 +- .../runners/spark/ForceStreamingTest.java | 18 +-- .../beam/runners/spark/PipelineRule.java | 109 ------------------ .../spark/ProvidedSparkContextTest.java | 10 +- .../spark/SparkRunnerDebuggerTest.java | 15 +-- .../spark/SparkRunnerRegistrarTest.java | 2 +- .../beam/runners/spark/StreamingTest.java | 23 ++++ .../metrics/sink/SparkMetricsSinkTest.java | 12 +- .../runners/spark/io/AvroPipelineTest.java | 10 +- .../beam/runners/spark/io/NumShardsTest.java | 6 +- .../spark/translation/StorageLevelTest.java | 31 ++++- .../streaming/CreateStreamTest.java | 53 ++++----- .../ResumeFromCheckpointStreamingTest.java | 50 +++++--- .../streaming/StreamingSourceMetricsTest.java | 14 +-- .../java/org/apache/beam/sdk/Pipeline.java | 2 +- .../apache/beam/sdk/testing/TestPipeline.java | 21 +++- 18 files changed, 217 insertions(+), 222 deletions(-) delete mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/PipelineRule.java create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/StreamingTest.java diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 38d250e7e733..f7200d6ed0cf 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -333,9 +333,6 @@ org.apache.maven.plugins maven-surefire-plugin - - org.apache.beam.runners.spark.UsesCheckpointRecovery - 1 false @@ -344,6 +341,50 @@ false + + + default-test + + test + + + + org.apache.beam.runners.spark.UsesCheckpointRecovery, + org.apache.beam.runners.spark.StreamingTest + + + + [ + "--runner=TestSparkRunner", + "--streaming=false", + "--enableSparkMetricSinks=true" + ] + + + + + + streaming-tests + test + + test + + + + org.apache.beam.runners.spark.StreamingTest + + + + [ + "--runner=TestSparkRunner", + "--forceStreaming=true", + "--enableSparkMetricSinks=true" + ] + + + + + org.codehaus.mojo diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java index bedfda419f41..bf926dc54ecc 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java @@ -54,7 +54,9 @@ public Iterable>> getPipelineRunners() { public static class Options implements PipelineOptionsRegistrar { @Override public Iterable> getPipelineOptions() { - return ImmutableList.>of(SparkPipelineOptions.class); + return ImmutableList.>of( + SparkPipelineOptions.class, + TestSparkPipelineOptions.class); } } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java index c3b48d82fb22..24b2e7b3a358 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java @@ -23,11 +23,11 @@ import org.apache.beam.runners.spark.translation.SparkContextFactory; import org.apache.beam.runners.spark.translation.TransformTranslator; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; import org.apache.spark.api.java.JavaSparkContext; -import org.junit.Rule; import org.junit.Test; /** @@ -36,12 +36,12 @@ */ public class CacheTest { - @Rule - public final transient PipelineRule pipelineRule = PipelineRule.batch(); - @Test public void cacheCandidatesUpdaterTest() throws Exception { - Pipeline pipeline = pipelineRule.createPipeline(); + SparkPipelineOptions options = + PipelineOptionsFactory.create().as(TestSparkPipelineOptions.class); + options.setRunner(TestSparkRunner.class); + Pipeline pipeline = Pipeline.create(options); PCollection pCollection = pipeline.apply(Create.of("foo", "bar")); // first read pCollection.apply(Count.globally()); @@ -50,7 +50,7 @@ public void cacheCandidatesUpdaterTest() throws Exception { // will cache the RDD representing this PCollection pCollection.apply(Count.globally()); - JavaSparkContext jsc = SparkContextFactory.getSparkContext(pipelineRule.getOptions()); + JavaSparkContext jsc = SparkContextFactory.getSparkContext(options); EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); SparkRunner.CacheVisitor cacheVisitor = new SparkRunner.CacheVisitor(new TransformTranslator.Translator(), ctxt); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java index b60faf2e238d..7bfc980304a3 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/ForceStreamingTest.java @@ -25,9 +25,9 @@ import org.apache.beam.sdk.io.BoundedReadFromUnboundedSource; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; -import org.junit.Rule; import org.junit.Test; @@ -44,19 +44,23 @@ */ public class ForceStreamingTest { - @Rule - public final PipelineRule pipelineRule = PipelineRule.streaming(); - @Test public void test() throws IOException { - Pipeline pipeline = pipelineRule.createPipeline(); + TestSparkPipelineOptions options = + PipelineOptionsFactory.create().as(TestSparkPipelineOptions.class); + options.setRunner(TestSparkRunner.class); + options.setForceStreaming(true); + + // pipeline with a bounded read. + Pipeline pipeline = Pipeline.create(options); // apply the BoundedReadFromUnboundedSource. BoundedReadFromUnboundedSource boundedRead = Read.from(CountingSource.unbounded()).withMaxNumRecords(-1); - //noinspection unchecked pipeline.apply(boundedRead); - TestSparkRunner runner = TestSparkRunner.fromOptions(pipelineRule.getOptions()); + + // adapt reads + TestSparkRunner runner = TestSparkRunner.fromOptions(options); runner.adaptBoundedReads(pipeline); UnboundedReadDetector unboundedReadDetector = new UnboundedReadDetector(); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/PipelineRule.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/PipelineRule.java deleted file mode 100644 index f8499f301e92..000000000000 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/PipelineRule.java +++ /dev/null @@ -1,109 +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.spark; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.joda.time.Duration; -import org.junit.rules.ExternalResource; -import org.junit.rules.RuleChain; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestName; -import org.junit.rules.TestRule; -import org.junit.runner.Description; -import org.junit.runners.model.Statement; - -/** - * A {@link org.junit.Rule} to provide a {@link Pipeline} instance for Spark runner tests. - */ -public class PipelineRule implements TestRule { - - private final SparkPipelineRule delegate; - private final RuleChain chain; - - private PipelineRule(SparkPipelineRule delegate) { - TestName testName = new TestName(); - this.delegate = delegate; - this.delegate.setTestName(testName); - this.chain = RuleChain.outerRule(testName).around(this.delegate); - } - - public static PipelineRule streaming() { - return new PipelineRule(new SparkStreamingPipelineRule()); - } - - public static PipelineRule batch() { - return new PipelineRule(new SparkPipelineRule()); - } - - public Duration batchDuration() { - return Duration.millis(delegate.options.getBatchIntervalMillis()); - } - - public TestSparkPipelineOptions getOptions() { - return delegate.options; - } - - public Pipeline createPipeline() { - return Pipeline.create(delegate.options); - } - - @Override - public Statement apply(Statement statement, Description description) { - return chain.apply(statement, description); - } - - private static class SparkStreamingPipelineRule extends SparkPipelineRule { - - private final TemporaryFolder temporaryFolder = new TemporaryFolder(); - - @Override - protected void before() throws Throwable { - super.before(); - temporaryFolder.create(); - options.setForceStreaming(true); - options.setCheckpointDir( - temporaryFolder.newFolder(options.getJobName()).toURI().toURL().toString()); - } - - @Override - protected void after() { - temporaryFolder.delete(); - } - } - - private static class SparkPipelineRule extends ExternalResource { - - protected final TestSparkPipelineOptions options = - PipelineOptionsFactory.as(TestSparkPipelineOptions.class); - - private TestName testName; - - public void setTestName(TestName testName) { - this.testName = testName; - } - - @Override - protected void before() throws Throwable { - options.setRunner(TestSparkRunner.class); - options.setEnableSparkMetricSinks(false); - options.setJobName( - testName != null ? testName.getMethodName() : "test-at-" + System.currentTimeMillis()); - } - } -} diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java index 36ba863e4272..811299308f26 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java @@ -27,9 +27,11 @@ import java.util.Set; import org.apache.beam.runners.spark.examples.WordCount; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.values.PCollection; @@ -95,7 +97,9 @@ private void testWithValidProvidedContext(JavaSparkContext jsc) throws Exception PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); // Run test from pipeline - p.run().waitUntilFinish(); + PipelineResult result = p.run(); + + TestPipeline.verifyPAssertsSucceeded(p, result); } private void testWithInvalidContext(JavaSparkContext jsc) { @@ -104,11 +108,9 @@ private void testWithInvalidContext(JavaSparkContext jsc) { Pipeline p = Pipeline.create(options); PCollection inputWords = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder .of())); - PCollection output = inputWords.apply(new WordCount.CountWords()) + inputWords.apply(new WordCount.CountWords()) .apply(MapElements.via(new WordCount.FormatAsTextFn())); - PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); - try { p.run().waitUntilFinish(); fail("Should throw an exception when The provided Spark context is null or stopped"); 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 ea058b2b749c..90097511c45b 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 @@ -27,6 +27,8 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Distinct; @@ -48,7 +50,6 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.hamcrest.Matchers; import org.joda.time.Duration; -import org.junit.Rule; import org.junit.Test; @@ -57,15 +58,9 @@ */ public class SparkRunnerDebuggerTest { - @Rule - public final PipelineRule batchPipelineRule = PipelineRule.batch(); - - @Rule - public final PipelineRule streamingPipelineRule = PipelineRule.streaming(); - @Test public void debugBatchPipeline() { - TestSparkPipelineOptions options = batchPipelineRule.getOptions(); + PipelineOptions options = PipelineOptionsFactory.create().as(TestSparkPipelineOptions.class); options.setRunner(SparkRunnerDebugger.class); Pipeline pipeline = Pipeline.create(options); @@ -111,7 +106,9 @@ public void debugBatchPipeline() { @Test public void debugStreamingPipeline() { - TestSparkPipelineOptions options = streamingPipelineRule.getOptions(); + TestSparkPipelineOptions options = + PipelineOptionsFactory.create().as(TestSparkPipelineOptions.class); + options.setForceStreaming(true); options.setRunner(SparkRunnerDebugger.class); Pipeline pipeline = Pipeline.create(options); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java index 4e1fd7c5afee..75899f9a395f 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java @@ -38,7 +38,7 @@ public class SparkRunnerRegistrarTest { @Test public void testOptions() { assertEquals( - ImmutableList.of(SparkPipelineOptions.class), + ImmutableList.of(SparkPipelineOptions.class, TestSparkPipelineOptions.class), new SparkRunnerRegistrar.Options().getPipelineOptions()); } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/StreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/StreamingTest.java new file mode 100644 index 000000000000..a34c184d3b66 --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/StreamingTest.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. + */ +package org.apache.beam.runners.spark; + +/** + * Category tag for tests that should be run in streaming mode. + */ +public interface StreamingTest {} diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/SparkMetricsSinkTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/SparkMetricsSinkTest.java index b0ad97253d74..fff95cbcaf11 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/SparkMetricsSinkTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/SparkMetricsSinkTest.java @@ -26,11 +26,10 @@ import java.util.Arrays; import java.util.List; import java.util.Set; -import org.apache.beam.runners.spark.PipelineRule; import org.apache.beam.runners.spark.examples.WordCount; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; 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.transforms.MapElements; import org.apache.beam.sdk.values.PCollection; @@ -48,12 +47,7 @@ public class SparkMetricsSinkTest { public ExternalResource inMemoryMetricsSink = new InMemoryMetricsSinkRule(); @Rule - public final PipelineRule pipelineRule = PipelineRule.batch(); - - private Pipeline createSparkPipeline() { - pipelineRule.getOptions().setEnableSparkMetricSinks(true); - return pipelineRule.createPipeline(); - } + public final TestPipeline pipeline = TestPipeline.create(); private void runPipeline() { final List words = @@ -62,8 +56,6 @@ private void runPipeline() { final Set expectedCounts = ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2"); - final Pipeline pipeline = createSparkPipeline(); - final PCollection output = pipeline .apply(Create.of(words).withCoder(StringUtf8Coder.of())) diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java index 7188dc598fbd..adde8d258c8b 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java @@ -33,9 +33,8 @@ import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; -import org.apache.beam.runners.spark.PipelineRule; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.junit.Before; import org.junit.Rule; @@ -54,7 +53,7 @@ public class AvroPipelineTest { public final TemporaryFolder tmpDir = new TemporaryFolder(); @Rule - public final PipelineRule pipelineRule = PipelineRule.batch(); + public final TestPipeline pipeline = TestPipeline.create(); @Before public void setUp() throws IOException { @@ -72,11 +71,10 @@ public void testGeneric() throws Exception { savedRecord.put("siblingnames", Lists.newArrayList("Jimmy", "Jane")); populateGenericFile(Lists.newArrayList(savedRecord), schema); - Pipeline p = pipelineRule.createPipeline(); - PCollection input = p.apply( + PCollection input = pipeline.apply( AvroIO.readGenericRecords(schema).from(inputFile.getAbsolutePath())); input.apply(AvroIO.writeGenericRecords(schema).to(outputDir.getAbsolutePath())); - p.run().waitUntilFinish(); + pipeline.run(); List records = readGenericFile(); assertEquals(Lists.newArrayList(savedRecord), records); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java index 50217447b0af..55ee93827541 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java @@ -30,11 +30,10 @@ import java.util.Arrays; import java.util.List; import java.util.Set; -import org.apache.beam.runners.spark.PipelineRule; import org.apache.beam.runners.spark.examples.WordCount; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.values.PCollection; @@ -59,7 +58,7 @@ public class NumShardsTest { public final TemporaryFolder tmpDir = new TemporaryFolder(); @Rule - public final PipelineRule pipelineRule = PipelineRule.batch(); + public final TestPipeline p = TestPipeline.create(); @Before public void setUp() throws IOException { @@ -69,7 +68,6 @@ public void setUp() throws IOException { @Test public void testText() throws Exception { - Pipeline p = pipelineRule.createPipeline(); PCollection inputWords = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder.of())); PCollection output = inputWords.apply(new WordCount.CountWords()) .apply(MapElements.via(new WordCount.FormatAsTextFn())); 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 2b7b87b11f09..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 @@ -15,30 +15,49 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.beam.runners.spark.translation; -import org.apache.beam.runners.spark.PipelineRule; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; + /** * Test the RDD storage level defined by user. */ public class StorageLevelTest { + private static String beamTestPipelineOptions; + @Rule - public final transient PipelineRule pipelineRule = PipelineRule.batch(); + public final TestPipeline pipeline = TestPipeline.create(); + + @BeforeClass + public static void init() { + beamTestPipelineOptions = + System.getProperty(TestPipeline.PROPERTY_BEAM_TEST_PIPELINE_OPTIONS); + + System.setProperty( + TestPipeline.PROPERTY_BEAM_TEST_PIPELINE_OPTIONS, + beamTestPipelineOptions.replace("]", ", \"--storageLevel=DISK_ONLY\"]")); + } + + @AfterClass + public static void teardown() { + System.setProperty( + TestPipeline.PROPERTY_BEAM_TEST_PIPELINE_OPTIONS, + beamTestPipelineOptions); + } @Test public void test() throws Exception { - pipelineRule.getOptions().setStorageLevel("DISK_ONLY"); - Pipeline pipeline = pipelineRule.createPipeline(); - PCollection pCollection = pipeline.apply(Create.of("foo")); // by default, the Spark runner doesn't cache the RDD if it accessed only one time. diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java index dd52c05985eb..770e0c054a92 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java @@ -24,13 +24,14 @@ import java.io.IOException; import java.io.Serializable; -import org.apache.beam.runners.spark.PipelineRule; import org.apache.beam.runners.spark.ReuseSparkContextRule; +import org.apache.beam.runners.spark.SparkPipelineOptions; +import org.apache.beam.runners.spark.StreamingTest; import org.apache.beam.runners.spark.io.CreateStream; -import org.apache.beam.sdk.Pipeline; 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.transforms.Combine; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; @@ -61,6 +62,7 @@ import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; @@ -74,10 +76,11 @@ * {@link org.apache.spark.streaming.dstream.QueueInputDStream} and advance the system's WMs. * //TODO: add synchronized/processing time trigger. */ +@Category(StreamingTest.class) public class CreateStreamTest implements Serializable { @Rule - public final transient PipelineRule pipelineRule = PipelineRule.streaming(); + public final transient TestPipeline p = TestPipeline.create(); @Rule public final transient ReuseSparkContextRule noContextResue = ReuseSparkContextRule.no(); @Rule @@ -85,10 +88,9 @@ public class CreateStreamTest implements Serializable { @Test public void testLateDataAccumulating() throws IOException { - Pipeline p = pipelineRule.createPipeline(); Instant instant = new Instant(0); CreateStream source = - CreateStream.of(VarIntCoder.of(), pipelineRule.batchDuration()) + CreateStream.of(VarIntCoder.of(), batchDuration()) .emptyBatch() .advanceWatermarkForNextBatch(instant.plus(Duration.standardMinutes(6))) .nextBatch( @@ -159,9 +161,8 @@ public Void apply(Iterable input) { @Test public void testDiscardingMode() throws IOException { - Pipeline p = pipelineRule.createPipeline(); CreateStream source = - CreateStream.of(StringUtf8Coder.of(), pipelineRule.batchDuration()) + CreateStream.of(StringUtf8Coder.of(), batchDuration()) .nextBatch( TimestampedValue.of("firstPane", new Instant(100)), TimestampedValue.of("alsoFirstPane", new Instant(200))) @@ -208,10 +209,9 @@ public void testDiscardingMode() throws IOException { @Test public void testFirstElementLate() throws IOException { - Pipeline p = pipelineRule.createPipeline(); Instant lateElementTimestamp = new Instant(-1_000_000); CreateStream source = - CreateStream.of(StringUtf8Coder.of(), pipelineRule.batchDuration()) + CreateStream.of(StringUtf8Coder.of(), batchDuration()) .emptyBatch() .advanceWatermarkForNextBatch(new Instant(0)) .nextBatch( @@ -242,10 +242,9 @@ public void testFirstElementLate() throws IOException { @Test public void testElementsAtAlmostPositiveInfinity() throws IOException { - Pipeline p = pipelineRule.createPipeline(); Instant endOfGlobalWindow = GlobalWindow.INSTANCE.maxTimestamp(); CreateStream source = - CreateStream.of(StringUtf8Coder.of(), pipelineRule.batchDuration()) + CreateStream.of(StringUtf8Coder.of(), batchDuration()) .nextBatch( TimestampedValue.of("foo", endOfGlobalWindow), TimestampedValue.of("bar", endOfGlobalWindow)) @@ -267,13 +266,12 @@ public void testElementsAtAlmostPositiveInfinity() throws IOException { @Test public void testMultipleStreams() throws IOException { - Pipeline p = pipelineRule.createPipeline(); CreateStream source = - CreateStream.of(StringUtf8Coder.of(), pipelineRule.batchDuration()) + CreateStream.of(StringUtf8Coder.of(), batchDuration()) .nextBatch("foo", "bar") .advanceNextBatchWatermarkToInfinity(); CreateStream other = - CreateStream.of(VarIntCoder.of(), pipelineRule.batchDuration()) + CreateStream.of(VarIntCoder.of(), batchDuration()) .nextBatch(1, 2, 3, 4) .advanceNextBatchWatermarkToInfinity(); @@ -298,10 +296,9 @@ public void testMultipleStreams() throws IOException { @Test public void testFlattenedWithWatermarkHold() throws IOException { - Pipeline p = pipelineRule.createPipeline(); Instant instant = new Instant(0); CreateStream source1 = - CreateStream.of(VarIntCoder.of(), pipelineRule.batchDuration()) + CreateStream.of(VarIntCoder.of(), batchDuration()) .emptyBatch() .advanceWatermarkForNextBatch(instant.plus(Duration.standardMinutes(5))) .nextBatch( @@ -310,7 +307,7 @@ public void testFlattenedWithWatermarkHold() throws IOException { TimestampedValue.of(3, instant)) .advanceWatermarkForNextBatch(instant.plus(Duration.standardMinutes(10))); CreateStream source2 = - CreateStream.of(VarIntCoder.of(), pipelineRule.batchDuration()) + CreateStream.of(VarIntCoder.of(), batchDuration()) .emptyBatch() .advanceWatermarkForNextBatch(instant.plus(Duration.standardMinutes(1))) .nextBatch( @@ -323,14 +320,14 @@ public void testFlattenedWithWatermarkHold() throws IOException { .advanceNextBatchWatermarkToInfinity(); PCollection windowed1 = p - .apply(source1) - .apply(Window.into(FixedWindows.of(Duration.standardMinutes(5))) + .apply("CreateStream1", source1) + .apply("Window1", Window.into(FixedWindows.of(Duration.standardMinutes(5))) .triggering(AfterWatermark.pastEndOfWindow()) .accumulatingFiredPanes() .withAllowedLateness(Duration.ZERO)); PCollection windowed2 = p - .apply(source2) - .apply(Window.into(FixedWindows.of(Duration.standardMinutes(5))) + .apply("CreateStream2", source2) + .apply("Window2", Window.into(FixedWindows.of(Duration.standardMinutes(5))) .triggering(AfterWatermark.pastEndOfWindow()) .accumulatingFiredPanes() .withAllowedLateness(Duration.ZERO)); @@ -357,10 +354,9 @@ public void testFlattenedWithWatermarkHold() throws IOException { */ @Test public void testMultiOutputParDo() throws IOException { - Pipeline p = pipelineRule.createPipeline(); Instant instant = new Instant(0); CreateStream source1 = - CreateStream.of(VarIntCoder.of(), pipelineRule.batchDuration()) + CreateStream.of(VarIntCoder.of(), batchDuration()) .emptyBatch() .advanceWatermarkForNextBatch(instant.plus(Duration.standardMinutes(5))) .nextBatch( @@ -397,7 +393,7 @@ public void process(ProcessContext context) { @Test public void testElementAtPositiveInfinityThrows() { CreateStream source = - CreateStream.of(VarIntCoder.of(), pipelineRule.batchDuration()) + CreateStream.of(VarIntCoder.of(), batchDuration()) .nextBatch(TimestampedValue.of(-1, BoundedWindow.TIMESTAMP_MAX_VALUE.minus(1L))) .advanceNextBatchWatermarkToInfinity(); thrown.expect(IllegalArgumentException.class); @@ -407,7 +403,7 @@ public void testElementAtPositiveInfinityThrows() { @Test public void testAdvanceWatermarkNonMonotonicThrows() { CreateStream source = - CreateStream.of(VarIntCoder.of(), pipelineRule.batchDuration()) + CreateStream.of(VarIntCoder.of(), batchDuration()) .advanceWatermarkForNextBatch(new Instant(0L)); thrown.expect(IllegalArgumentException.class); source @@ -418,9 +414,14 @@ public void testAdvanceWatermarkNonMonotonicThrows() { @Test public void testAdvanceWatermarkEqualToPositiveInfinityThrows() { CreateStream source = - CreateStream.of(VarIntCoder.of(), pipelineRule.batchDuration()) + CreateStream.of(VarIntCoder.of(), batchDuration()) .advanceWatermarkForNextBatch(BoundedWindow.TIMESTAMP_MAX_VALUE.minus(1L)); thrown.expect(IllegalArgumentException.class); source.advanceWatermarkForNextBatch(BoundedWindow.TIMESTAMP_MAX_VALUE); } + + private Duration batchDuration() { + return Duration.millis( + (p.getOptions().as(SparkPipelineOptions.class)).getBatchIntervalMillis()); + } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java index 33571f0f1691..584edace250a 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java @@ -28,15 +28,16 @@ import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Uninterruptibles; import java.io.IOException; +import java.io.Serializable; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.concurrent.TimeUnit; -import org.apache.beam.runners.spark.PipelineRule; import org.apache.beam.runners.spark.ReuseSparkContextRule; import org.apache.beam.runners.spark.SparkPipelineResult; import org.apache.beam.runners.spark.TestSparkPipelineOptions; +import org.apache.beam.runners.spark.TestSparkRunner; import org.apache.beam.runners.spark.UsesCheckpointRecovery; import org.apache.beam.runners.spark.aggregators.AggregatorsAccumulator; import org.apache.beam.runners.spark.io.MicrobatchSource; @@ -53,6 +54,7 @@ import org.apache.beam.sdk.metrics.MetricResult; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.metrics.MetricsFilter; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; @@ -81,11 +83,12 @@ import org.joda.time.Instant; import org.junit.After; import org.junit.AfterClass; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; - +import org.junit.rules.TemporaryFolder; /** * Tests DStream recovery from checkpoint. @@ -96,24 +99,34 @@ * {@link Metrics} values that are expected to resume from previous count and a side-input that is * expected to recover as well. */ -public class ResumeFromCheckpointStreamingTest { +public class ResumeFromCheckpointStreamingTest implements Serializable { private static final EmbeddedKafkaCluster.EmbeddedZookeeper EMBEDDED_ZOOKEEPER = new EmbeddedKafkaCluster.EmbeddedZookeeper(); private static final EmbeddedKafkaCluster EMBEDDED_KAFKA_CLUSTER = new EmbeddedKafkaCluster(EMBEDDED_ZOOKEEPER.getConnection(), new Properties()); private static final String TOPIC = "kafka_beam_test_topic"; + private transient TemporaryFolder temporaryFolder; + @Rule public final transient ReuseSparkContextRule noContextReuse = ReuseSparkContextRule.no(); - @Rule - public final transient PipelineRule pipelineRule = PipelineRule.streaming(); @BeforeClass - public static void init() throws IOException { + public static void setup() throws IOException { EMBEDDED_ZOOKEEPER.startup(); EMBEDDED_KAFKA_CLUSTER.startup(); } + @Before + public void init() { + temporaryFolder = new TemporaryFolder(); + try { + temporaryFolder.create(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + private static void produce(Map messages) { Properties producerProps = new Properties(); producerProps.putAll(EMBEDDED_KAFKA_CLUSTER.getProps()); @@ -148,7 +161,7 @@ public void testWithResume() throws Exception { .build(); // first run should expect EOT matching the last injected element. - SparkPipelineResult res = run(pipelineRule, Optional.of(new Instant(400)), 0); + SparkPipelineResult res = run(Optional.of(new Instant(400)), 0); assertThat(res.metrics().queryMetrics(metricsFilter).counters(), hasItem(attemptedMetricsResult(ResumeFromCheckpointStreamingTest.class.getName(), @@ -169,7 +182,7 @@ public void testWithResume() throws Exception { )); // recovery should resume from last read offset, and read the second batch of input. - res = runAgain(pipelineRule, 1); + res = runAgain(1); // assertions 2: assertThat(res.metrics().queryMetrics(metricsFilter).counters(), hasItem(attemptedMetricsResult(ResumeFromCheckpointStreamingTest.class.getName(), @@ -209,18 +222,18 @@ public void testWithResume() throws Exception { String.format("Found %d failed assertions.", failedAssertions), failedAssertions, is(0L)); - } - private SparkPipelineResult runAgain(PipelineRule pipelineRule, int expectedAssertions) { + private SparkPipelineResult runAgain(int expectedAssertions) { // sleep before next run. Uninterruptibles.sleepUninterruptibly(10, TimeUnit.MILLISECONDS); - return run(pipelineRule, Optional.absent(), expectedAssertions); + return run(Optional.absent(), expectedAssertions); } @SuppressWarnings("OptionalUsedAsFieldOrParameterType") - private static SparkPipelineResult run( - PipelineRule pipelineRule, Optional stopWatermarkOption, int expectedAssertions) { + private SparkPipelineResult run( + Optional stopWatermarkOption, + int expectedAssertions) { KafkaIO.Read read = KafkaIO.read() .withBootstrapServers(EMBEDDED_KAFKA_CLUSTER.getBrokerList()) .withTopics(Collections.singletonList(TOPIC)) @@ -242,15 +255,21 @@ public Instant apply(KV kv) { } }); - TestSparkPipelineOptions options = pipelineRule.getOptions(); + TestSparkPipelineOptions options = + PipelineOptionsFactory.create().as(TestSparkPipelineOptions.class); options.setSparkMaster("local[*]"); options.setCheckpointDurationMillis(options.getBatchIntervalMillis()); options.setExpectedAssertions(expectedAssertions); + options.setRunner(TestSparkRunner.class); + options.setEnableSparkMetricSinks(false); + options.setForceStreaming(true); + options.setCheckpointDir(temporaryFolder.getRoot().getPath()); // timeout is per execution so it can be injected by the caller. if (stopWatermarkOption.isPresent()) { options.setStopPipelineWatermark(stopWatermarkOption.get().getMillis()); } - Pipeline p = pipelineRule.createPipeline(); + + Pipeline p = Pipeline.create(options); PCollection expectedCol = p.apply(Create.of(ImmutableList.of("side1", "side2")).withCoder(StringUtf8Coder.of())); @@ -354,5 +373,4 @@ public void processElement(ProcessContext c) throws Exception { } } } - } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StreamingSourceMetricsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StreamingSourceMetricsTest.java index 5a4b1b5e2919..df6027cf349b 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StreamingSourceMetricsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StreamingSourceMetricsTest.java @@ -23,9 +23,7 @@ import static org.junit.Assert.assertThat; import java.io.Serializable; -import org.apache.beam.runners.spark.PipelineRule; -import org.apache.beam.runners.spark.TestSparkPipelineOptions; -import org.apache.beam.sdk.Pipeline; +import org.apache.beam.runners.spark.StreamingTest; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.Source; @@ -34,10 +32,11 @@ import org.apache.beam.sdk.metrics.MetricQueryResults; import org.apache.beam.sdk.metrics.MetricsFilter; import org.apache.beam.sdk.metrics.SourceMetrics; +import org.apache.beam.sdk.testing.TestPipeline; import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; - +import org.junit.experimental.categories.Category; /** * Verify metrics support for {@link Source Sources} in streaming pipelines. @@ -47,14 +46,11 @@ public class StreamingSourceMetricsTest implements Serializable { // Force streaming pipeline using pipeline rule. @Rule - public final transient PipelineRule pipelineRule = PipelineRule.streaming(); + public final transient TestPipeline pipeline = TestPipeline.create(); @Test + @Category(StreamingTest.class) public void testUnboundedSourceMetrics() { - TestSparkPipelineOptions options = pipelineRule.getOptions(); - - Pipeline pipeline = Pipeline.create(options); - final long numElements = 1000; pipeline.apply( 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 6b15f0d3f655..6fa7a5a3b4e5 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 @@ -460,7 +460,7 @@ OutputT applyTransform(String name, InputT input, private Set usedFullNames = new HashSet<>(); private CoderRegistry coderRegistry; private final List unstableNames = new ArrayList<>(); - private final PipelineOptions defaultOptions; + protected final PipelineOptions defaultOptions; protected Pipeline(PipelineOptions options) { this.defaultOptions = options; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index d8fe51d92c72..2d34b2249467 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Predicate; import com.google.common.base.Predicates; @@ -244,8 +245,11 @@ public static class PipelineRunMissingException extends RuntimeException { } } - static final String PROPERTY_BEAM_TEST_PIPELINE_OPTIONS = "beamTestPipelineOptions"; + /** System property used to set {@link TestPipelineOptions}. */ + public static final String PROPERTY_BEAM_TEST_PIPELINE_OPTIONS = "beamTestPipelineOptions"; + static final String PROPERTY_USE_DEFAULT_DUMMY_RUNNER = "beamUseDummyRunner"; + private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( ObjectMapper.findModules(ReflectHelpers.findClassLoader())); @@ -331,7 +335,7 @@ public PipelineResult run() { try { enforcement.get().beforePipelineExecution(); pipelineResult = super.run(); - verifyPAssertsSucceeded(pipelineResult); + verifyPAssertsSucceeded(this, pipelineResult); } catch (RuntimeException exc) { Throwable cause = exc.getCause(); if (cause instanceof AssertionError) { @@ -377,6 +381,15 @@ public TestPipeline enableAutoRunIfMissing(final boolean enable) { return this; } + @VisibleForTesting + @Override + /** + * Get this pipeline's options. + */ + public PipelineOptions getOptions() { + return defaultOptions; + } + @Override public String toString() { return "TestPipeline#" + getOptions().as(ApplicationNameOptions.class).getAppName(); @@ -501,9 +514,9 @@ private static Optional findCallersStackTrace() { *

Note this only runs for runners which support Metrics. Runners which do not should verify * this in some other way. See: https://issues.apache.org/jira/browse/BEAM-2001

*/ - private void verifyPAssertsSucceeded(PipelineResult pipelineResult) { + public static void verifyPAssertsSucceeded(Pipeline pipeline, PipelineResult pipelineResult) { if (MetricsEnvironment.isMetricsSupported()) { - long expectedNumberOfAssertions = (long) PAssert.countAsserts(this); + long expectedNumberOfAssertions = (long) PAssert.countAsserts(pipeline); long successfulAssertions = 0; Iterable> successCounterResults = From 731f9d9fbb466baea793a9e7376ce790a0cca98b Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 09:33:41 -0700 Subject: [PATCH 036/387] GCP-IO: manage versions in top level pom --- pom.xml | 21 +++++++++++++++++++++ sdks/java/io/google-cloud-platform/pom.xml | 7 ------- 2 files changed, 21 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index 127babdadaf5..15428913d9fa 100644 --- a/pom.xml +++ b/pom.xml @@ -106,6 +106,7 @@ 2.24.0 1.8.1 v2-rev295-1.22.0 + 0.9.6 v1-rev6-1.22.0 0.1.0 v2-rev8-1.22.0 @@ -129,6 +130,7 @@ 4.12 1.9.5 4.1.8.Final + 1.1.33.Fork26 1.5.0.Final 3.2.0 v1-rev10-1.22.0 @@ -797,6 +799,25 @@
+ + com.google.cloud.bigtable + bigtable-protos + ${bigtable.version} + + + + com.google.cloud.bigtable + bigtable-client-core + ${bigtable.version} + + + + io.netty + netty-tcnative-boringssl-static + ${netty.tcnative.version} + runtime + + com.google.cloud.datastore datastore-v1-proto-client diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 9051d987f738..9c1286c4119e 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -31,10 +31,6 @@ IO library to read and write Google Cloud Platform systems from Beam. jar - - 0.9.6 - - @@ -158,13 +154,11 @@ com.google.cloud.bigtable bigtable-protos - ${bigtable.version} com.google.cloud.bigtable bigtable-client-core - ${bigtable.version} @@ -220,7 +214,6 @@ io.netty netty-tcnative-boringssl-static - 1.1.33.Fork26 runtime From 40dc844304a8432ee5c8e81b9cc806ef2ae3c1ea Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Wed, 3 May 2017 22:41:06 -0700 Subject: [PATCH 037/387] Remove IOChannelUtil/Factory from BigQueryIO --- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 2 - .../sdk/io/gcp/bigquery/BigQueryHelpers.java | 17 +++----- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 35 ++++++++------- .../io/gcp/bigquery/BigQuerySourceBase.java | 14 +++--- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 43 +++++-------------- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 15 +++---- .../sdk/io/gcp/bigquery/FakeJobService.java | 29 ++++++++----- 7 files changed, 65 insertions(+), 90 deletions(-) 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 78d39b5090fc..f4221354621b 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 @@ -205,7 +205,6 @@ public String apply(String input) { bigQueryServices, jobIdTokenView, schemasView, - stepUuid, WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED, dynamicDestinations)) @@ -243,7 +242,6 @@ public String apply(String input) { bigQueryServices, jobIdTokenView, schemasView, - stepUuid, writeDisposition, createDisposition, dynamicDestinations)) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java index 6b4e518a5c50..318ea8910b8d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java @@ -36,12 +36,12 @@ import java.util.regex.Matcher; import javax.annotation.Nullable; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.ResolveOptions; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.IOChannelFactory; -import org.apache.beam.sdk.util.IOChannelUtils; /** A set of helper functions and classes used by {@link BigQueryIO}. */ public class BigQueryHelpers { @@ -309,14 +309,9 @@ static TableReference createTempTableReference(String projectId, String jobUuid) static String resolveTempLocation( String tempLocationDir, String bigQueryOperationName, String stepUuid) { - try { - IOChannelFactory factory = IOChannelUtils.getFactory(tempLocationDir); - return factory.resolve( - factory.resolve(tempLocationDir, bigQueryOperationName), stepUuid); - } catch (IOException e) { - throw new RuntimeException( - String.format("Failed to resolve temp destination directory in %s", - tempLocationDir), e); - } + return FileSystems.matchNewResource(tempLocationDir, true) + .resolve(bigQueryOperationName, ResolveOptions.StandardResolveOptions.RESOLVE_DIRECTORY) + .resolve(stepUuid, ResolveOptions.StandardResolveOptions.RESOLVE_DIRECTORY) + .toString(); } } 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 c76ee86948d1..0e36393ea31e 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 @@ -38,8 +38,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; + import java.io.IOException; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.regex.Pattern; @@ -50,6 +50,10 @@ import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.MoveOptions; +import org.apache.beam.sdk.io.fs.ResolveOptions; +import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonTableRefToTableRef; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableRefToJson; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSchemaToJsonSchema; @@ -67,8 +71,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.IOChannelFactory; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.Transport; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; @@ -520,19 +522,14 @@ void cleanup(PipelineOptions options) throws Exception { Job extractJob = getBigQueryServices().getJobService(bqOptions).getJob(jobRef); - Collection extractFiles = null; if (extractJob != null) { - extractFiles = getExtractFilePaths(extractDestinationDir, extractJob); - } else { - IOChannelFactory factory = IOChannelUtils.getFactory(extractDestinationDir); - Collection dirMatch = factory.match(extractDestinationDir); - if (!dirMatch.isEmpty()) { - extractFiles = factory.match(factory.resolve(extractDestinationDir, "*")); + List extractFiles = + getExtractFilePaths(extractDestinationDir, extractJob); + if (extractFiles != null && !extractFiles.isEmpty()) { + FileSystems.delete(extractFiles, + MoveOptions.StandardMoveOptions.IGNORE_MISSING_FILES); } } - if (extractFiles != null && !extractFiles.isEmpty()) { - IOChannelUtils.getFactory(extractFiles.iterator().next()).remove(extractFiles); - } } }; return input.getPipeline() @@ -583,7 +580,7 @@ static String getExtractDestinationUri(String extractDestinationDir) { return String.format("%s/%s", extractDestinationDir, "*.avro"); } - static List getExtractFilePaths(String extractDestinationDir, Job extractJob) + static List getExtractFilePaths(String extractDestinationDir, Job extractJob) throws IOException { JobStatistics jobStats = extractJob.getStatistics(); List counts = jobStats.getExtract().getDestinationUriFileCounts(); @@ -597,11 +594,13 @@ static List getExtractFilePaths(String extractDestinationDir, Job extrac } long filesCount = counts.get(0); - ImmutableList.Builder paths = ImmutableList.builder(); - IOChannelFactory factory = IOChannelUtils.getFactory(extractDestinationDir); + ImmutableList.Builder paths = ImmutableList.builder(); + ResourceId extractDestinationDirResourceId = + FileSystems.matchNewResource(extractDestinationDir, true /* isDirectory */); for (long i = 0; i < filesCount; ++i) { - String filePath = - factory.resolve(extractDestinationDir, String.format("%012d%s", i, ".avro")); + ResourceId filePath = extractDestinationDirResourceId.resolve( + String.format("%012d%s", i, ".avro"), + ResolveOptions.StandardResolveOptions.RESOLVE_FILE); paths.add(filePath); } return paths.build(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java index 49000d610924..945c7d40b19b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java @@ -38,6 +38,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.AvroSource; import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.Status; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; import org.apache.beam.sdk.options.PipelineOptions; @@ -90,7 +91,7 @@ public List> split( resolveTempLocation(bqOptions.getTempLocation(), "BigQueryExtractTemp", stepUuid); String extractJobId = getExtractJobId(createJobIdToken(options.getJobName(), stepUuid)); - List tempFiles = executeExtract( + List tempFiles = executeExtract( extractJobId, tableToExtract, jobService, bqOptions.getProject(), extractDestinationDir); TableSchema tableSchema = bqServices.getDatasetService(bqOptions) @@ -116,7 +117,7 @@ public Coder getDefaultOutputCoder() { return TableRowJsonCoder.of(); } - private List executeExtract( + private List executeExtract( String jobId, TableReference table, JobService jobService, String executingProject, String extractDestinationDir) throws InterruptedException, IOException { @@ -143,12 +144,11 @@ private List executeExtract( LOG.info("BigQuery extract job completed: {}", jobId); - List tempFiles = BigQueryIO.getExtractFilePaths(extractDestinationDir, extractJob); - return ImmutableList.copyOf(tempFiles); + return BigQueryIO.getExtractFilePaths(extractDestinationDir, extractJob); } private List> createSources( - List files, TableSchema tableSchema) throws IOException, InterruptedException { + List files, TableSchema tableSchema) throws IOException, InterruptedException { final String jsonSchema = BigQueryIO.JSON_FACTORY.toString(tableSchema); SerializableFunction function = @@ -160,9 +160,9 @@ public TableRow apply(GenericRecord input) { }}; List> avroSources = Lists.newArrayList(); - for (String fileName : files) { + for (ResourceId file : files) { avroSources.add(new TransformingSource<>( - AvroSource.from(fileName), function, getDefaultOutputCoder())); + AvroSource.from(file.toString()), function, getDefaultOutputCoder())); } return ImmutableList.copyOf(avroSources); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index c480b421c6b3..e7dba2a87a96 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -18,37 +18,30 @@ package org.apache.beam.sdk.io.gcp.bigquery; -import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.resolveTempLocation; - import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationLoad; import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Paths; import java.util.Collection; import java.util.List; import java.util.Map; import javax.annotation.Nullable; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.MoveOptions; +import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.Status; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.FileIOChannelFactory; -import org.apache.beam.sdk.util.GcsIOChannelFactory; -import org.apache.beam.sdk.util.GcsUtil; -import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory; -import org.apache.beam.sdk.util.IOChannelFactory; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.slf4j.Logger; @@ -74,7 +67,6 @@ class WriteTables private final BigQueryServices bqServices; private final PCollectionView jobIdToken; private final PCollectionView> schemasView; - private final String stepUuid; private final WriteDisposition writeDisposition; private final CreateDisposition createDisposition; private final DynamicDestinations dynamicDestinations; @@ -84,7 +76,6 @@ public WriteTables( BigQueryServices bqServices, PCollectionView jobIdToken, PCollectionView> schemasView, - String stepUuid, WriteDisposition writeDisposition, CreateDisposition createDisposition, DynamicDestinations dynamicDestinations) { @@ -92,7 +83,6 @@ public WriteTables( this.bqServices = bqServices; this.jobIdToken = jobIdToken; this.schemasView = schemasView; - this.stepUuid = stepUuid; this.writeDisposition = writeDisposition; this.createDisposition = createDisposition; this.dynamicDestinations = dynamicDestinations; @@ -114,8 +104,6 @@ public void processElement(ProcessContext c) throws Exception { tableReference, tableDestination.getTableDescription()); } - String tempFilePrefix = resolveTempLocation( - c.getPipelineOptions().getTempLocation(), "BigQueryWriteTemp", stepUuid); Integer partition = c.element().getKey().getShardNumber(); List partitionFiles = Lists.newArrayList(c.element().getValue()); String jobIdPrefix = @@ -137,7 +125,7 @@ public void processElement(ProcessContext c) throws Exception { tableDestination.getTableDescription()); c.output(KV.of(tableDestination, BigQueryHelpers.toJsonString(tableReference))); - removeTemporaryFiles(c.getPipelineOptions(), tempFilePrefix, partitionFiles); + removeTemporaryFiles(partitionFiles); } private void load( @@ -198,22 +186,11 @@ private void load( BigQueryHelpers.jobToPrettyString(lastFailedLoadJob))); } - static void removeTemporaryFiles( - PipelineOptions options, String tempFilePrefix, Collection files) throws IOException { - IOChannelFactory factory = IOChannelUtils.getFactory(tempFilePrefix); - if (factory instanceof GcsIOChannelFactory) { - GcsUtil gcsUtil = new GcsUtilFactory().create(options); - gcsUtil.remove(files); - } else if (factory instanceof FileIOChannelFactory) { - for (String filename : files) { - LOG.debug("Removing file {}", filename); - boolean exists = Files.deleteIfExists(Paths.get(filename)); - if (!exists) { - LOG.debug("{} does not exist.", filename); - } - } - } else { - throw new IOException("Unrecognized file system."); + static void removeTemporaryFiles(Collection files) throws IOException { + ImmutableList.Builder fileResources = ImmutableList.builder(); + for (String file: files) { + fileResources.add(FileSystems.matchNewResource(file, false/* isDirectory */)); } + FileSystems.delete(fileResources.build(), MoveOptions.StandardMoveOptions.IGNORE_MISSING_FILES); } } 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 026afce4b3a6..aabae3e963c6 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 @@ -83,7 +83,9 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.CountingSource; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonSchemaToTableSchema; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; @@ -118,7 +120,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.WindowedValue; @@ -1820,7 +1821,9 @@ public void testWriteTables() throws Exception { for (int k = 0; k < numFilesPerPartition; ++k) { String filename = Paths.get(baseDir.toString(), String.format("files0x%08x_%05d", tempTableId.hashCode(), k)).toString(); - try (WritableByteChannel channel = IOChannelUtils.create(filename, MimeTypes.TEXT)) { + ResourceId fileResource = + FileSystems.matchNewResource(filename, false /* isDirectory */); + try (WritableByteChannel channel = FileSystems.create(fileResource, MimeTypes.TEXT)) { try (OutputStream output = Channels.newOutputStream(channel)) { TableRow tableRow = new TableRow().set("name", tableName); TableRowJsonCoder.of().encode(tableRow, output, Context.OUTER); @@ -1858,7 +1861,6 @@ public void testWriteTables() throws Exception { fakeBqServices, jobIdTokenView, schemaMapView, - stepUuid, WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED, new IdentityDynamicTables()); @@ -1904,14 +1906,9 @@ public void testRemoveTemporaryFiles() throws Exception { File tempDir = new File(bqOptions.getTempLocation()); testNumFiles(tempDir, 10); - WriteTables.removeTemporaryFiles(bqOptions, tempFilePrefix, fileNames); + WriteTables.removeTemporaryFiles(fileNames); testNumFiles(tempDir, 0); - - for (String fileName : fileNames) { - loggedWriteTables.verifyDebug("Removing file " + fileName); - } - loggedWriteTables.verifyDebug(fileNames.get(numFiles) + " does not exist."); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java index 13d345ea7944..ee3af0ba9892 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java @@ -40,6 +40,7 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.HashBasedTable; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import java.io.BufferedReader; @@ -61,12 +62,14 @@ import org.apache.avro.generic.GenericRecordBuilder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.MoveOptions; +import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.Transport; import org.joda.time.Duration; @@ -95,7 +98,7 @@ private static class JobInfo { HashBasedTable.create(); private static int numExtractJobCalls = 0; - private static final com.google.common.collect.Table> + private static final com.google.common.collect.Table> filesForLoadJobs = HashBasedTable.create(); private static final com.google.common.collect.Table dryRunQueryResults = HashBasedTable.create(); @@ -117,12 +120,17 @@ public void startLoadJob(JobReference jobRef, JobConfigurationLoad loadConfig) // Copy the files to a new location for import, as the temporary files will be deleted by // the caller. if (loadConfig.getSourceUris().size() > 0) { - List loadFiles = Lists.newArrayList(); + ImmutableList.Builder sourceFiles = ImmutableList.builder(); + ImmutableList.Builder loadFiles = ImmutableList.builder(); for (String filename : loadConfig.getSourceUris()) { - loadFiles.add(filename + ThreadLocalRandom.current().nextInt()); + sourceFiles.add(FileSystems.matchNewResource(filename, false /* isDirectory */)); + loadFiles.add(FileSystems.matchNewResource( + filename + ThreadLocalRandom.current().nextInt(), false /* isDirectory */)); } - IOChannelUtils.getFactory(loadFiles.get(0)).copy(loadConfig.getSourceUris(), loadFiles); - filesForLoadJobs.put(jobRef.getProjectId(), jobRef.getJobId(), loadFiles); + + FileSystems.copy(sourceFiles.build(), loadFiles.build(), + MoveOptions.StandardMoveOptions.IGNORE_MISSING_FILES); + filesForLoadJobs.put(jobRef.getProjectId(), jobRef.getJobId(), loadFiles.build()); } allJobs.put(jobRef.getProjectId(), jobRef.getJobId(), new JobInfo(job)); @@ -286,7 +294,7 @@ private JobStatus runLoadJob(JobReference jobRef, JobConfigurationLoad load) throws InterruptedException, IOException { TableReference destination = load.getDestinationTable(); TableSchema schema = load.getSchema(); - List sourceFiles = filesForLoadJobs.get(jobRef.getProjectId(), jobRef.getJobId()); + List sourceFiles = filesForLoadJobs.get(jobRef.getProjectId(), jobRef.getJobId()); WriteDisposition writeDisposition = WriteDisposition.valueOf(load.getWriteDisposition()); CreateDisposition createDisposition = CreateDisposition.valueOf(load.getCreateDisposition()); checkArgument(load.getSourceFormat().equals("NEWLINE_DELIMITED_JSON")); @@ -298,8 +306,8 @@ private JobStatus runLoadJob(JobReference jobRef, JobConfigurationLoad load) datasetService.createTable(new Table().setTableReference(destination).setSchema(schema)); List rows = Lists.newArrayList(); - for (String filename : sourceFiles) { - rows.addAll(readRows(filename)); + for (ResourceId filename : sourceFiles) { + rows.addAll(readRows(filename.toString())); } datasetService.insertAll(destination, rows, null); return new JobStatus().setState("DONE"); @@ -385,7 +393,8 @@ private long writeRows(String tableId, List rows, TableSchema schema, private void writeRowsHelper(List rows, Schema avroSchema, String destinationPattern, int shard) throws IOException { String filename = destinationPattern.replace("*", String.format("%012d", shard)); - try (WritableByteChannel channel = IOChannelUtils.create(filename, MimeTypes.BINARY); + try (WritableByteChannel channel = FileSystems.create( + FileSystems.matchNewResource(filename, false /* isDirectory */), MimeTypes.BINARY); DataFileWriter tableRowWriter = new DataFileWriter<>(new GenericDatumWriter(avroSchema)) .create(avroSchema, Channels.newOutputStream(channel))) { From 55351dcebec8ba9e166c4f90555edca6b90b1b14 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 3 May 2017 03:06:36 -0700 Subject: [PATCH 038/387] Eliminate Pipeline.getOptions --- .../examples/complete/game/GameStats.java | 7 ++- .../complete/game/HourlyTeamScore.java | 6 ++- .../examples/complete/game/LeaderBoard.java | 26 +++++++---- .../examples/complete/game/UserScore.java | 20 ++++++--- .../complete/game/utils/WriteToBigQuery.java | 32 ++++++++------ .../game/utils/WriteWindowedToBigQuery.java | 8 ++-- .../beam/runners/direct/DirectRunner.java | 1 + .../runners/direct/DisplayDataValidator.java | 6 +-- .../dataflow/testing/TestDataflowRunner.java | 3 +- .../testing/TestDataflowRunnerTest.java | 18 +++----- .../beam/runners/spark/SparkRunner.java | 4 +- .../runners/spark/SparkRunnerDebugger.java | 4 +- .../beam/runners/spark/TestSparkRunner.java | 5 +-- .../spark/translation/EvaluationContext.java | 16 +++++-- .../translation/SparkRuntimeContext.java | 4 +- .../SparkRunnerStreamingContextFactory.java | 2 +- .../apache/beam/runners/spark/CacheTest.java | 2 +- .../streaming/TrackStreamingSourcesTest.java | 2 +- .../java/org/apache/beam/sdk/Pipeline.java | 14 +----- .../apache/beam/sdk/testing/TestPipeline.java | 26 +++++------ .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 44 +++++++++---------- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 1 + 22 files changed, 131 insertions(+), 120 deletions(-) diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java index a46d3c533768..abbb13bf93ee 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java @@ -24,6 +24,7 @@ import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; @@ -300,6 +301,8 @@ public void processElement(ProcessContext c) { // Write the result to BigQuery .apply("WriteTeamSums", new WriteWindowedToBigQuery>( + options.as(GcpOptions.class).getProject(), + options.getDataset(), options.getGameStatsTablePrefix() + "_team", configureWindowedWrite())); @@ -327,7 +330,9 @@ public void processElement(ProcessContext c) { // Write this info to a BigQuery table. .apply("WriteAvgSessionLength", new WriteWindowedToBigQuery( - options.getGameStatsTablePrefix() + "_sessions", configureSessionWindowWrite())); + options.as(GcpOptions.class).getProject(), + options.getDataset(), + options.getGameStatsTablePrefix() + "_sessions", configureSessionWindowWrite())); // [END DocInclude_Rewindow] diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java index 3f1ffb0de56c..2928882a1776 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java @@ -22,6 +22,7 @@ import java.util.TimeZone; import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -185,7 +186,10 @@ public static void main(String[] args) throws Exception { // Extract and sum teamname/score pairs from the event data. .apply("ExtractTeamScore", new ExtractAndSumScore("team")) .apply("WriteTeamScoreSums", - new WriteWindowedToBigQuery>(options.getHourlyTeamScoreTableName(), + new WriteWindowedToBigQuery>( + options.as(GcpOptions.class).getProject(), + options.getDataset(), + options.getHourlyTeamScoreTableName(), configureWindowedTableWrite())); diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java index 9af34c536e80..bfad9f65fc3f 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java @@ -27,6 +27,7 @@ import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -194,14 +195,20 @@ public static void main(String[] args) throws Exception { .withTimestampAttribute(TIMESTAMP_ATTRIBUTE).fromTopic(options.getTopic())) .apply("ParseGameEvent", ParDo.of(new ParseEventFn())); - gameEvents.apply("CalculateTeamScores", - new CalculateTeamScores( - Duration.standardMinutes(options.getTeamWindowDuration()), - Duration.standardMinutes(options.getAllowedLateness()))) + gameEvents + .apply( + "CalculateTeamScores", + new CalculateTeamScores( + Duration.standardMinutes(options.getTeamWindowDuration()), + Duration.standardMinutes(options.getAllowedLateness()))) // Write the results to BigQuery. - .apply("WriteTeamScoreSums", - new WriteWindowedToBigQuery>( - options.getLeaderBoardTableName() + "_team", configureWindowedTableWrite())); + .apply( + "WriteTeamScoreSums", + new WriteWindowedToBigQuery>( + options.as(GcpOptions.class).getProject(), + options.getDataset(), + options.getLeaderBoardTableName() + "_team", + configureWindowedTableWrite())); gameEvents .apply( "CalculateUserScores", @@ -210,7 +217,10 @@ public static void main(String[] args) throws Exception { .apply( "WriteUserScoreSums", new WriteToBigQuery>( - options.getLeaderBoardTableName() + "_user", configureGlobalWindowBigQueryWrite())); + options.as(GcpOptions.class).getProject(), + options.getDataset(), + options.getLeaderBoardTableName() + "_user", + configureGlobalWindowBigQueryWrite())); // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the // command line. diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java index c136c2e3de38..8110146d7020 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/UserScore.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; @@ -226,13 +227,18 @@ public static void main(String[] args) throws Exception { Pipeline pipeline = Pipeline.create(options); // Read events from a text file and parse them. - pipeline.apply(TextIO.read().from(options.getInput())) - .apply("ParseGameEvent", ParDo.of(new ParseEventFn())) - // Extract and sum username/score pairs from the event data. - .apply("ExtractUserScore", new ExtractAndSumScore("user")) - .apply("WriteUserScoreSums", - new WriteToBigQuery>(options.getUserScoreTableName(), - configureBigQueryWrite())); + pipeline + .apply(TextIO.read().from(options.getInput())) + .apply("ParseGameEvent", ParDo.of(new ParseEventFn())) + // Extract and sum username/score pairs from the event data. + .apply("ExtractUserScore", new ExtractAndSumScore("user")) + .apply( + "WriteUserScoreSums", + new WriteToBigQuery>( + options.as(GcpOptions.class).getProject(), + options.getDataset(), + options.getUserScoreTableName(), + configureBigQueryWrite())); // Run the batch pipeline. pipeline.run().waitUntilFinish(); diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java index f767d21b6fb5..2ec4e5c9a133 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java @@ -25,13 +25,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import org.apache.beam.examples.complete.game.UserScore; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; -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; @@ -47,14 +43,21 @@ public class WriteToBigQuery extends PTransform, PDone> { + protected String projectId; + protected String datasetId; protected String tableName; protected Map> fieldInfo; public WriteToBigQuery() { } - public WriteToBigQuery(String tableName, + public WriteToBigQuery( + String projectId, + String datasetId, + String tableName, Map> fieldInfo) { + this.projectId = projectId; + this.datasetId = datasetId; this.tableName = tableName; this.fieldInfo = fieldInfo; } @@ -120,20 +123,21 @@ protected TableSchema getSchema() { @Override public PDone expand(PCollection teamAndScore) { teamAndScore - .apply("ConvertToRow", ParDo.of(new BuildRowFn())) - .apply(BigQueryIO.writeTableRows().to(getTable(teamAndScore.getPipeline(), tableName)) - .withSchema(getSchema()) - .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) - .withWriteDisposition(WriteDisposition.WRITE_APPEND)); + .apply("ConvertToRow", ParDo.of(new BuildRowFn())) + .apply( + BigQueryIO.writeTableRows() + .to(getTable(projectId, datasetId, tableName)) + .withSchema(getSchema()) + .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(WriteDisposition.WRITE_APPEND)); return PDone.in(teamAndScore.getPipeline()); } /** Utility to construct an output table reference. */ - static TableReference getTable(Pipeline pipeline, String tableName) { - PipelineOptions options = pipeline.getOptions(); + static TableReference getTable(String projectId, String datasetId, String tableName) { TableReference table = new TableReference(); - table.setDatasetId(options.as(UserScore.Options.class).getDataset()); - table.setProjectId(options.as(GcpOptions.class).getProject()); + table.setDatasetId(datasetId); + table.setProjectId(projectId); table.setTableId(tableName); return table; } diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java index e602258c74fa..deb9db2c0fd5 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java @@ -37,9 +37,9 @@ public class WriteWindowedToBigQuery extends WriteToBigQuery { - public WriteWindowedToBigQuery(String tableName, - Map> fieldInfo) { - super(tableName, fieldInfo); + public WriteWindowedToBigQuery( + String projectId, String datasetId, String tableName, Map> fieldInfo) { + super(projectId, datasetId, tableName, fieldInfo); } /** Convert each key/score pair into a BigQuery TableRow. */ @@ -62,7 +62,7 @@ public PDone expand(PCollection teamAndScore) { teamAndScore .apply("ConvertToRow", ParDo.of(new BuildRowFn())) .apply(BigQueryIO.writeTableRows() - .to(getTable(teamAndScore.getPipeline(), tableName)) + .to(getTable(projectId, datasetId, tableName)) .withSchema(getSchema()) .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) .withWriteDisposition(WriteDisposition.WRITE_APPEND)); 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 c6168b3e6603..984598a6faaf 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 @@ -163,6 +163,7 @@ public DirectPipelineResult run(Pipeline pipeline) { pipeline.traverseTopologically(keyedPValueVisitor); DisplayDataValidator.validatePipeline(pipeline); + DisplayDataValidator.validateOptions(getPipelineOptions()); DirectGraph graph = graphVisitor.getGraph(); EvaluationContext context = diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java index c77cb48f4f9b..209c8010bcab 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.direct; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; @@ -31,12 +32,11 @@ class DisplayDataValidator { private DisplayDataValidator() {} static void validatePipeline(Pipeline pipeline) { - validateOptions(pipeline); validateTransforms(pipeline); } - private static void validateOptions(Pipeline pipeline) { - evaluateDisplayData(pipeline.getOptions()); + static void validateOptions(PipelineOptions options) { + evaluateDisplayData(options); } private static void validateTransforms(Pipeline pipeline) { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java index ba9d971646b3..c238d80bc933 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java @@ -98,7 +98,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { DataflowPipelineJob run(Pipeline pipeline, DataflowRunner runner) { updatePAssertCount(pipeline); - TestPipelineOptions testPipelineOptions = pipeline.getOptions().as(TestPipelineOptions.class); + TestPipelineOptions testPipelineOptions = options.as(TestPipelineOptions.class); final DataflowPipelineJob job; job = runner.run(pipeline); @@ -188,7 +188,6 @@ DataflowPipelineJob run(Pipeline pipeline, DataflowRunner runner) { @VisibleForTesting void updatePAssertCount(Pipeline pipeline) { - DataflowPipelineOptions options = pipeline.getOptions().as(DataflowPipelineOptions.class); if (DataflowRunner.hasExperiment(options, "beam_fn_api")) { // TODO[BEAM-1866]: FnAPI does not support metrics, so expect 0 assertions. expectedNumberOfAssertions = 0; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java index 54eb88d04f87..eb068e67c3b3 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunnerTest.java @@ -464,8 +464,7 @@ public void testBatchOnCreateMatcher() throws Exception { when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); - p.getOptions().as(TestPipelineOptions.class) - .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0)); + options.as(TestPipelineOptions.class).setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0)); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(true /* success */, true /* tentative */)); @@ -488,8 +487,7 @@ public void testStreamingOnCreateMatcher() throws Exception { when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); - p.getOptions().as(TestPipelineOptions.class) - .setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0)); + options.as(TestPipelineOptions.class).setOnCreateMatcher(new TestSuccessMatcher(mockJob, 0)); when(mockJob.waitUntilFinish(any(Duration.class), any(JobMessagesHandler.class))) .thenReturn(State.DONE); @@ -515,8 +513,7 @@ public void testBatchOnSuccessMatcherWhenPipelineSucceeds() throws Exception { when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); - p.getOptions().as(TestPipelineOptions.class) - .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1)); + options.as(TestPipelineOptions.class).setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1)); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(true /* success */, true /* tentative */)); @@ -544,8 +541,7 @@ public void testStreamingOnSuccessMatcherWhenPipelineSucceeds() throws Exception when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); - p.getOptions().as(TestPipelineOptions.class) - .setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1)); + options.as(TestPipelineOptions.class).setOnSuccessMatcher(new TestSuccessMatcher(mockJob, 1)); when(mockJob.waitUntilFinish(any(Duration.class), any(JobMessagesHandler.class))) .thenReturn(State.DONE); @@ -570,8 +566,7 @@ public void testBatchOnSuccessMatcherWhenPipelineFails() throws Exception { when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); - p.getOptions().as(TestPipelineOptions.class) - .setOnSuccessMatcher(new TestFailureMatcher()); + options.as(TestPipelineOptions.class).setOnSuccessMatcher(new TestFailureMatcher()); when(mockClient.getJobMetrics(anyString())) .thenReturn(generateMockMetricResponse(false /* success */, true /* tentative */)); @@ -606,8 +601,7 @@ public void testStreamingOnSuccessMatcherWhenPipelineFails() throws Exception { when(mockRunner.run(any(Pipeline.class))).thenReturn(mockJob); TestDataflowRunner runner = TestDataflowRunner.fromOptionsAndClient(options, mockClient); - p.getOptions().as(TestPipelineOptions.class) - .setOnSuccessMatcher(new TestFailureMatcher()); + options.as(TestPipelineOptions.class).setOnSuccessMatcher(new TestFailureMatcher()); when(mockJob.waitUntilFinish(any(Duration.class), any(JobMessagesHandler.class))) .thenReturn(State.FAILED); 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 97487f3f0277..1a0c04234da2 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 @@ -193,7 +193,7 @@ public void run() { } else { // create the evaluation context final JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions); - final EvaluationContext evaluationContext = new EvaluationContext(jsc, pipeline); + final EvaluationContext evaluationContext = new EvaluationContext(jsc, pipeline, mOptions); translator = new TransformTranslator.Translator(); // update the cache candidates @@ -383,7 +383,7 @@ protected boolean shouldDefer(TransformHierarchy.Node node) { LOG.info( "Deferring combine transformation {} for job {}", transform, - ctxt.getPipeline().getOptions().getJobName()); + ctxt.getOptions().getJobName()); return true; } // default. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java index 7f7aefc2b6b6..8d47e1a31e82 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java @@ -89,10 +89,10 @@ public SparkPipelineResult run(Pipeline pipeline) { && ((TestSparkPipelineOptions) options).isForceStreaming()) { SparkPipelineTranslator streamingTranslator = new StreamingTransformTranslator.Translator(translator); - EvaluationContext ctxt = new EvaluationContext(jsc, pipeline, jssc); + EvaluationContext ctxt = new EvaluationContext(jsc, pipeline, options, jssc); visitor = new SparkNativePipelineVisitor(streamingTranslator, ctxt); } else { - EvaluationContext ctxt = new EvaluationContext(jsc, pipeline, jssc); + EvaluationContext ctxt = new EvaluationContext(jsc, pipeline, options, jssc); visitor = new SparkNativePipelineVisitor(translator, ctxt); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index 1e678131d92b..6808d7b6b132 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -83,6 +83,7 @@ public final class TestSparkRunner extends PipelineRunner { private static final Logger LOG = LoggerFactory.getLogger(TestSparkRunner.class); + private final TestSparkPipelineOptions testSparkPipelineOptions; private SparkRunner delegate; private boolean isForceStreaming; @@ -90,6 +91,7 @@ public final class TestSparkRunner extends PipelineRunner { private TestSparkRunner(TestSparkPipelineOptions options) { this.delegate = SparkRunner.fromOptions(options); this.isForceStreaming = options.isForceStreaming(); + this.testSparkPipelineOptions = options; } public static TestSparkRunner fromOptions(PipelineOptions options) { @@ -101,9 +103,6 @@ public static TestSparkRunner fromOptions(PipelineOptions options) { @Override public SparkPipelineResult run(Pipeline pipeline) { - TestSparkPipelineOptions testSparkPipelineOptions = - pipeline.getOptions().as(TestSparkPipelineOptions.class); - // // if the pipeline forces execution as a streaming pipeline, // and the source is an adapted unbounded source (as bounded), // read it as unbounded source via UnboundedReadFromBoundedSource. 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 838c504a1aad..5d77e9162e7d 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 @@ -30,6 +30,7 @@ import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; @@ -57,15 +58,18 @@ public class EvaluationContext { private AppliedPTransform currentTransform; private final SparkPCollectionView pviews = new SparkPCollectionView(); private final Map cacheCandidates = new HashMap<>(); + private final PipelineOptions options; - public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) { + public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline, PipelineOptions options) { this.jsc = jsc; this.pipeline = pipeline; - this.runtime = new SparkRuntimeContext(pipeline); + this.options = options; + this.runtime = new SparkRuntimeContext(pipeline, options); } - public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline, JavaStreamingContext jssc) { - this(jsc, pipeline); + public EvaluationContext( + JavaSparkContext jsc, Pipeline pipeline, PipelineOptions options, JavaStreamingContext jssc) { + this(jsc, pipeline, options); this.jssc = jssc; } @@ -81,6 +85,10 @@ public Pipeline getPipeline() { return pipeline; } + public PipelineOptions getOptions() { + return options; + } + public SparkRuntimeContext getRuntimeContext() { return runtime; } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java index e006143cc0f3..3db1ab51ddbb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java @@ -35,8 +35,8 @@ public class SparkRuntimeContext implements Serializable { private final String serializedPipelineOptions; private transient CoderRegistry coderRegistry; - SparkRuntimeContext(Pipeline pipeline) { - this.serializedPipelineOptions = serializePipelineOptions(pipeline.getOptions()); + SparkRuntimeContext(Pipeline pipeline, PipelineOptions options) { + this.serializedPipelineOptions = serializePipelineOptions(options); } private String serializePipelineOptions(PipelineOptions pipelineOptions) { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java index 2dd18f395b0c..6a153ffd6d40 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java @@ -82,7 +82,7 @@ public JavaStreamingContext call() throws Exception { // We must first init accumulators since translators expect them to be instantiated. SparkRunner.initAccumulators(options, jsc); - EvaluationContext ctxt = new EvaluationContext(jsc, pipeline, jssc); + EvaluationContext ctxt = new EvaluationContext(jsc, pipeline, options, jssc); // update cache candidates SparkRunner.updateCacheCandidates(pipeline, translator, ctxt); pipeline.traverseTopologically(new SparkRunner.Evaluator(translator, ctxt)); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java index 24b2e7b3a358..d3d0823c0470 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java @@ -51,7 +51,7 @@ public void cacheCandidatesUpdaterTest() throws Exception { pCollection.apply(Count.globally()); JavaSparkContext jsc = SparkContextFactory.getSparkContext(options); - EvaluationContext ctxt = new EvaluationContext(jsc, pipeline); + EvaluationContext ctxt = new EvaluationContext(jsc, pipeline, options); SparkRunner.CacheVisitor cacheVisitor = new SparkRunner.CacheVisitor(new TransformTranslator.Translator(), ctxt); pipeline.traverseTopologically(cacheVisitor); 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 41ccd0837dcb..3dcab26e946b 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 @@ -135,7 +135,7 @@ private StreamingSourceTracker( Pipeline pipeline, Class transformClassToAssert, Integer... expected) { - this.ctxt = new EvaluationContext(jssc.sparkContext(), pipeline, jssc); + this.ctxt = new EvaluationContext(jssc.sparkContext(), pipeline, options, jssc); this.evaluator = new SparkRunner.Evaluator( new StreamingTransformTranslator.Translator(new TransformTranslator.Translator()), ctxt); this.transformClassToAssert = transformClassToAssert; 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 6fa7a5a3b4e5..f7c3f24c46bd 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 @@ -460,7 +460,7 @@ OutputT applyTransform(String name, InputT input, private Set usedFullNames = new HashSet<>(); private CoderRegistry coderRegistry; private final List unstableNames = new ArrayList<>(); - protected final PipelineOptions defaultOptions; + private final PipelineOptions defaultOptions; protected Pipeline(PipelineOptions options) { this.defaultOptions = options; @@ -471,18 +471,6 @@ public String toString() { return "Pipeline#" + hashCode(); } - /** - * Returns the default {@link PipelineOptions} provided to {@link #create(PipelineOptions)}. - * - * @deprecated see BEAM-818 Remove Pipeline.getPipelineOptions. Configuration should be explicitly - * provided to a transform if it is required. - */ - @Deprecated - public PipelineOptions getOptions() { - return defaultOptions; - } - - /** * Applies a {@link PTransform} to the given {@link PInput}. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index 2d34b2249467..96cae51765e6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -26,7 +26,6 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Predicate; import com.google.common.base.Predicates; @@ -104,6 +103,8 @@ */ public class TestPipeline extends Pipeline implements TestRule { + private final PipelineOptions options; + private static class PipelineRunEnforcement { @SuppressWarnings("WeakerAccess") @@ -183,10 +184,7 @@ private boolean isEmptyPipeline(final Pipeline pipeline) { private void verifyPipelineExecution() { if (!isEmptyPipeline(pipeline)) { if (!runAttempted && !enableAutoRunIfMissing) { - throw new PipelineRunMissingException( - "The pipeline has not been run (runner: " - + pipeline.getOptions().getRunner().getSimpleName() - + ")"); + throw new PipelineRunMissingException("The pipeline has not been run."); } else { final List pipelineNodes = recordPipelineNodes(pipeline); @@ -272,6 +270,11 @@ public static TestPipeline fromOptions(PipelineOptions options) { private TestPipeline(final PipelineOptions options) { super(options); + this.options = options; + } + + public PipelineOptions getOptions() { + return this.options; } @Override @@ -288,7 +291,7 @@ private void setDeducedEnforcementLevel() { .anyMatch(Annotations.Predicates.isCategoryOf(NeedsRunner.class, true)); final boolean crashingRunner = - CrashingRunner.class.isAssignableFrom(getOptions().getRunner()); + CrashingRunner.class.isAssignableFrom(options.getRunner()); checkState( !(annotatedWithNeedsRunner && crashingRunner), @@ -381,18 +384,9 @@ public TestPipeline enableAutoRunIfMissing(final boolean enable) { return this; } - @VisibleForTesting - @Override - /** - * Get this pipeline's options. - */ - public PipelineOptions getOptions() { - return defaultOptions; - } - @Override public String toString() { - return "TestPipeline#" + getOptions().as(ApplicationNameOptions.class).getAppName(); + return "TestPipeline#" + options.as(ApplicationNameOptions.class).getAppName(); } /** Creates {@link PipelineOptions} for testing. */ 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 0e36393ea31e..fbbf862d1e84 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 @@ -888,6 +888,26 @@ Write withTestServices(BigQueryServices testServices) { public void validate(PipelineOptions pipelineOptions) { BigQueryOptions options = pipelineOptions.as(BigQueryOptions.class); + // The user specified a table. + if (getJsonTableRef() != null && getJsonTableRef().isAccessible() && getValidate()) { + TableReference table = getTableWithDefaultProject(options).get(); + DatasetService datasetService = getBigQueryServices().getDatasetService(options); + // Check for destination table presence and emptiness for early failure notification. + // Note that a presence check can fail when the table or dataset is created by an earlier + // stage of the pipeline. For these cases the #withoutValidation method can be used to + // disable the check. + BigQueryHelpers.verifyDatasetPresence(datasetService, table); + if (getCreateDisposition() == BigQueryIO.Write.CreateDisposition.CREATE_NEVER) { + BigQueryHelpers.verifyTablePresence(datasetService, table); + } + if (getWriteDisposition() == BigQueryIO.Write.WriteDisposition.WRITE_EMPTY) { + BigQueryHelpers.verifyTableNotExistOrEmpty(datasetService, table); + } + } + } + + @Override + public WriteResult expand(PCollection input) { // We must have a destination to write to! checkState( getTableFunction() != null || getJsonTableRef() != null @@ -916,29 +936,7 @@ public void validate(PipelineOptions pipelineOptions) { checkArgument(2 > Iterables.size(Iterables.filter(allSchemaArgs, Predicates.notNull())), "No more than one of jsonSchema, schemaFromView, or dynamicDestinations may " - + "be set"); - - // The user specified a table. - if (getJsonTableRef() != null && getJsonTableRef().isAccessible() && getValidate()) { - TableReference table = getTableWithDefaultProject(options).get(); - DatasetService datasetService = getBigQueryServices().getDatasetService(options); - // Check for destination table presence and emptiness for early failure notification. - // Note that a presence check can fail when the table or dataset is created by an earlier - // stage of the pipeline. For these cases the #withoutValidation method can be used to - // disable the check. - BigQueryHelpers.verifyDatasetPresence(datasetService, table); - if (getCreateDisposition() == BigQueryIO.Write.CreateDisposition.CREATE_NEVER) { - BigQueryHelpers.verifyTablePresence(datasetService, table); - } - if (getWriteDisposition() == BigQueryIO.Write.WriteDisposition.WRITE_EMPTY) { - BigQueryHelpers.verifyTableNotExistOrEmpty(datasetService, table); - } - } - } - - @Override - public WriteResult expand(PCollection input) { - validate(input.getPipeline().getOptions()); + + "be set"); DynamicDestinations dynamicDestinations = getDynamicDestinations(); if (dynamicDestinations == null) { 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 aabae3e963c6..b893ad5a5189 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 @@ -1175,6 +1175,7 @@ public TableRow apply(Long input) { .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) .withSchema(new TableSchema()) .withTestServices(fakeBqServices)); + p.run(); } @Test From 7f19e7a0653e6e7f0d53ab537bc4b00f0ab486c0 Mon Sep 17 00:00:00 2001 From: Sourabh Bajaj Date: Thu, 4 May 2017 12:01:53 -0700 Subject: [PATCH 039/387] Cleanup old cython files before testing again --- sdks/python/tox.ini | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index 9c40eb5af8ce..2592b17501ae 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -48,6 +48,11 @@ deps = whitelist_externals=find commands = python --version + # Clean up all previous cython generated files. + - find apache_beam -type f -name '*.c' -delete + - find apache_beam -type f -name '*.so' -delete + - find target/build -type f -name '*.c' -delete + - find target/build -type f -name '*.so' -delete pip install -e .[test] python apache_beam/examples/complete/autocomplete_test.py python setup.py test From 4f934923d28798dfe7cd18c86ff4bcf8eebc27e5 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Mon, 20 Mar 2017 12:12:31 +0800 Subject: [PATCH 040/387] [BEAM-1727] Add align and offset to Timer --- .../construction/PTransformMatchersTest.java | 2 +- .../beam/runners/core/SimpleDoFnRunner.java | 44 ++++++- .../runners/core/SimpleDoFnRunnerTest.java | 2 +- .../org/apache/beam/sdk/transforms/DoFn.java | 2 +- .../java/org/apache/beam/sdk/util/Timer.java | 27 +++-- .../apache/beam/sdk/transforms/ParDoTest.java | 113 +++++++++++++++++- 6 files changed, 171 insertions(+), 19 deletions(-) diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java index e7d4c64805c0..bb1b1cd63184 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java @@ -193,7 +193,7 @@ public void processElement(ProcessContext c, @StateId(stateId) ValueState extends DoFn { * public void processElement( * ProcessContext c, * {@literal @TimerId("my-timer-id") Timer myTimer}) { - * myTimer.setForNowPlus(Duration.standardSeconds(...)); + * myTimer.offset(Duration.standardSeconds(...)).setRelative(); * } * * {@literal @OnTimer("my-timer-id")} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timer.java index 45a2a6623297..9727969fcc28 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timer.java @@ -49,19 +49,30 @@ public interface Timer { * *

For {@link TimeDomain#PROCESSING_TIME}, the behavior is be unpredictable, since processing * time timers are ignored after a window has expired. Instead, it is recommended to use - * {@link #setForNowPlus(Duration)}. + * {@link #setRelative()}. */ - void set(Instant instant); - - /** - * Sets or resets the time relative to the current time in the timer's {@link TimeDomain} at which - * this it should fire. If the timer was already set, resets it to the new requested time. - */ - void setForNowPlus(Duration durationFromNow); + void set(Instant absoluteTime); /** * Unsets this timer. It is permitted to {@code cancel()} whether or not the timer was actually * set. */ void cancel(); + + /** + * Sets the timer relative to the current time, according to any offset and alignment specified. + * Using {@link #offset(Duration)} and {@link #align(Duration)}. + */ + void setRelative(); + + /** + * Set the align offset. + */ + Timer offset(Duration offset); + + /** + * Aligns a timestamp to the next boundary of {@code period}. + */ + Timer align(Duration period); + } 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 d4475c925266..1c919d4fb659 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 @@ -2582,7 +2582,7 @@ public void testEventTimeTimerBounded() throws Exception { @ProcessElement public void processElement(ProcessContext context, @TimerId(timerId) Timer timer) { - timer.setForNowPlus(Duration.standardSeconds(1)); + timer.offset(Duration.standardSeconds(1)).setRelative(); context.output(3); } @@ -2597,6 +2597,36 @@ public void onTimer(OnTimerContext context) { pipeline.run(); } + @Test + @Category({ValidatesRunner.class, UsesTimersInParDo.class}) + public void testEventTimeTimerAlignBounded() throws Exception { + final String timerId = "foo"; + + DoFn, KV> fn = + new DoFn, KV>() { + + @TimerId(timerId) + private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @ProcessElement + public void processElement(ProcessContext context, @TimerId(timerId) Timer timer) { + timer.align(Duration.standardSeconds(1)).offset(Duration.millis(1)).setRelative(); + context.output(KV.of(3, context.timestamp())); + } + + @OnTimer(timerId) + public void onTimer(OnTimerContext context) { + context.output(KV.of(42, context.timestamp())); + } + }; + + PCollection> output = + pipeline.apply(Create.of(KV.of("hello", 37))).apply(ParDo.of(fn)); + PAssert.that(output).containsInAnyOrder(KV.of(3, BoundedWindow.TIMESTAMP_MIN_VALUE), + KV.of(42, BoundedWindow.TIMESTAMP_MIN_VALUE.plus(1774))); + pipeline.run(); + } + @Test @Category({ValidatesRunner.class, UsesTimersInParDo.class}) public void testTimerReceivedInOriginalWindow() throws Exception { @@ -2610,7 +2640,7 @@ public void testTimerReceivedInOriginalWindow() throws Exception { @ProcessElement public void processElement(ProcessContext context, @TimerId(timerId) Timer timer) { - timer.setForNowPlus(Duration.standardSeconds(1)); + timer.offset(Duration.standardSeconds(1)).setRelative(); } @OnTimer(timerId) @@ -2814,7 +2844,7 @@ public void testSimpleProcessingTimerTimer() throws Exception { @ProcessElement public void processElement(ProcessContext context, @TimerId(timerId) Timer timer) { - timer.setForNowPlus(Duration.standardSeconds(1)); + timer.offset(Duration.standardSeconds(1)).setRelative(); context.output(3); } @@ -2848,7 +2878,7 @@ public void testEventTimeTimerUnbounded() throws Exception { @ProcessElement public void processElement(ProcessContext context, @TimerId(timerId) Timer timer) { - timer.setForNowPlus(Duration.standardSeconds(1)); + timer.offset(Duration.standardSeconds(1)).setRelative(); context.output(3); } @@ -2870,6 +2900,81 @@ public void onTimer(OnTimerContext context) { pipeline.run(); } + @Test + @Category({NeedsRunner.class, UsesTimersInParDo.class, UsesTestStream.class}) + public void testEventTimeTimerAlignUnbounded() throws Exception { + final String timerId = "foo"; + + DoFn, KV> fn = + new DoFn, KV>() { + + @TimerId(timerId) + private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @ProcessElement + public void processElement(ProcessContext context, @TimerId(timerId) Timer timer) { + timer.align(Duration.standardSeconds(1)).offset(Duration.millis(1)).setRelative(); + context.output(KV.of(3, context.timestamp())); + } + + @OnTimer(timerId) + public void onTimer(OnTimerContext context) { + context.output(KV.of(42, context.timestamp())); + } + }; + + TestStream> stream = TestStream.create(KvCoder + .of(StringUtf8Coder.of(), VarIntCoder.of())) + .advanceWatermarkTo(new Instant(5)) + .addElements(KV.of("hello", 37)) + .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1).plus(1))) + .advanceWatermarkToInfinity(); + + PCollection> output = pipeline.apply(stream).apply(ParDo.of(fn)); + PAssert.that(output).containsInAnyOrder(KV.of(3, new Instant(5)), + KV.of(42, new Instant(Duration.standardSeconds(1).minus(1).getMillis()))); + pipeline.run(); + } + + @Test + @Category({NeedsRunner.class, UsesTimersInParDo.class, UsesTestStream.class}) + public void testEventTimeTimerAlignAfterGcTimeUnbounded() throws Exception { + final String timerId = "foo"; + + DoFn, KV> fn = + new DoFn, KV>() { + + @TimerId(timerId) + private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + + @ProcessElement + public void processElement(ProcessContext context, @TimerId(timerId) Timer timer) { + // This aligned time will exceed the END_OF_GLOBAL_WINDOW + timer.align(Duration.standardDays(1)).setRelative(); + context.output(KV.of(3, context.timestamp())); + } + + @OnTimer(timerId) + public void onTimer(OnTimerContext context) { + context.output(KV.of(42, context.timestamp())); + } + }; + + TestStream> stream = TestStream.create(KvCoder + .of(StringUtf8Coder.of(), VarIntCoder.of())) + // See GlobalWindow, + // END_OF_GLOBAL_WINDOW is TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(1)) + .advanceWatermarkTo(BoundedWindow.TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(1))) + .addElements(KV.of("hello", 37)) + .advanceWatermarkToInfinity(); + + PCollection> output = pipeline.apply(stream).apply(ParDo.of(fn)); + PAssert.that(output).containsInAnyOrder( + KV.of(3, BoundedWindow.TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(1))), + KV.of(42, BoundedWindow.TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(1)))); + pipeline.run(); + } + @Test public void testWithOutputTagsDisplayData() { DoFn fn = new DoFn() { From 7ece1647d6aeab4544d994a3a00360919920a978 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Tue, 2 May 2017 17:36:46 +0800 Subject: [PATCH 041/387] Fix initial watermark of DoFnOperator in Flink runner --- .../wrappers/streaming/DoFnOperator.java | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 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 16bf5d253419..518d6bed308d 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 @@ -198,8 +198,8 @@ protected DoFn getDoFn() { public void open() throws Exception { super.open(); - currentInputWatermark = Long.MIN_VALUE; - currentOutputWatermark = Long.MIN_VALUE; + setCurrentInputWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); + setCurrentOutputWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); sideInputReader = NullSideInputReader.of(sideInputs); @@ -429,18 +429,18 @@ public void processWatermark(Watermark mark) throws Exception { @Override public void processWatermark1(Watermark mark) throws Exception { if (keyCoder == null) { - this.currentInputWatermark = mark.getTimestamp(); + setCurrentInputWatermark(mark.getTimestamp()); long potentialOutputWatermark = Math.min(getPushbackWatermarkHold(), currentInputWatermark); if (potentialOutputWatermark > currentOutputWatermark) { - currentOutputWatermark = potentialOutputWatermark; + setCurrentOutputWatermark(potentialOutputWatermark); output.emitWatermark(new Watermark(currentOutputWatermark)); } } else { // fireTimers, so we need startBundle. pushbackDoFnRunner.startBundle(); - this.currentInputWatermark = mark.getTimestamp(); + setCurrentInputWatermark(mark.getTimestamp()); // hold back by the pushed back values waiting for side inputs long actualInputWatermark = Math.min(getPushbackWatermarkHold(), mark.getTimestamp()); @@ -454,7 +454,7 @@ public void processWatermark1(Watermark mark) throws Exception { long potentialOutputWatermark = Math.min(currentInputWatermark, combinedWatermarkHold); if (potentialOutputWatermark > currentOutputWatermark) { - currentOutputWatermark = potentialOutputWatermark; + setCurrentOutputWatermark(potentialOutputWatermark); output.emitWatermark(new Watermark(currentOutputWatermark)); } pushbackDoFnRunner.finishBundle(); @@ -608,6 +608,14 @@ public void fireTimer(InternalTimer timer) { timerData.getTimestamp(), timerData.getDomain()); } + private void setCurrentInputWatermark(long currentInputWatermark) { + this.currentInputWatermark = currentInputWatermark; + } + + private void setCurrentOutputWatermark(long currentOutputWatermark) { + this.currentOutputWatermark = currentOutputWatermark; + } + /** * Factory for creating an {@link DoFnRunners.OutputManager} from * a Flink {@link Output}. From 78ae52870dd906c63cc3574a6ec0e8e8fa8e510e Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 12:27:44 -0700 Subject: [PATCH 042/387] [BEAM-59] Remove IOChannelUtils, IOChannelFactory Leaving one deprecated function in one deprecated class to enable migration of DataflowRunner. --- .../beam/sdk/util/FileIOChannelFactory.java | 223 --------------- .../util/FileIOChannelFactoryRegistrar.java | 38 --- .../beam/sdk/util/IOChannelFactory.java | 127 --------- .../sdk/util/IOChannelFactoryRegistrar.java | 48 ---- .../apache/beam/sdk/util/IOChannelUtils.java | 259 +----------------- .../FileIOChannelFactoryRegistrarTest.java | 44 --- .../sdk/util/FileIOChannelFactoryTest.java | 242 ---------------- .../beam/sdk/util/IOChannelUtilsTest.java | 118 -------- .../beam/sdk/util/GcsIOChannelFactory.java | 111 -------- .../util/GcsIOChannelFactoryRegistrar.java | 38 --- .../GcsIOChannelFactoryRegistrarTest.java | 44 --- .../sdk/util/GcsIOChannelFactoryTest.java | 43 --- 12 files changed, 3 insertions(+), 1332 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactoryRegistrar.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactoryRegistrar.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryRegistrarTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java delete mode 100644 sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java delete mode 100644 sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java delete mode 100644 sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java delete mode 100644 sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java deleted file mode 100644 index 6f6ba3732602..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java +++ /dev/null @@ -1,223 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import java.io.BufferedOutputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.net.URI; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; -import java.nio.file.FileSystems; -import java.nio.file.Files; -import java.nio.file.NoSuchFileException; -import java.nio.file.Path; -import java.nio.file.PathMatcher; -import java.nio.file.Paths; -import java.nio.file.StandardCopyOption; -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; -import java.util.regex.Matcher; -import javax.annotation.Nullable; - -import org.apache.beam.sdk.options.PipelineOptions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Implements {@link IOChannelFactory} for local files. - */ -public class FileIOChannelFactory implements IOChannelFactory { - private static final Logger LOG = LoggerFactory.getLogger(FileIOChannelFactory.class); - - /** - * Create a {@link FileIOChannelFactory} with the given {@link PipelineOptions}. - */ - public static FileIOChannelFactory fromOptions(@Nullable PipelineOptions options) { - return new FileIOChannelFactory(); - } - - private FileIOChannelFactory() {} - - /** - * Converts the given file spec to a java {@link File}. If {@code spec} is actually a URI with - * the {@code file} scheme, then this function will ensure that the returned {@link File} - * has the correct path. - */ - private static File specToFile(String spec) { - try { - // Handle URI. - URI uri = URI.create(spec); - return Paths.get(uri).toFile(); - } catch (IllegalArgumentException e) { - // Fall back to assuming this is actually a file. - return Paths.get(spec).toFile(); - } - } - - /** - * {@inheritDoc} - * - *

Wildcards in the directory portion are not supported. - */ - @Override - public Collection match(String spec) throws IOException { - File file = specToFile(spec); - - File parent = file.getAbsoluteFile().getParentFile(); - if (!parent.exists()) { - return Collections.EMPTY_LIST; - } - - // Method getAbsolutePath() on Windows platform may return something like - // "c:\temp\file.txt". FileSystem.getPathMatcher() call below will treat - // '\' (backslash) as an escape character, instead of a directory - // separator. Replacing backslash with double-backslash solves the problem. - // We perform the replacement on all platforms, even those that allow - // backslash as a part of the filename, because Globs.toRegexPattern will - // eat one backslash. - String pathToMatch = file.getAbsolutePath().replaceAll(Matcher.quoteReplacement("\\"), - Matcher.quoteReplacement("\\\\")); - - final PathMatcher matcher = FileSystems.getDefault().getPathMatcher("glob:" + pathToMatch); - - Iterable files = com.google.common.io.Files.fileTreeTraverser().preOrderTraversal(parent); - Iterable matchedFiles = Iterables.filter(files, - Predicates.and( - com.google.common.io.Files.isFile(), - new Predicate() { - @Override - public boolean apply(File input) { - return matcher.matches(input.toPath()); - } - })); - - List result = new LinkedList<>(); - for (File match : matchedFiles) { - result.add(match.getPath()); - } - - return result; - } - - @Override - public ReadableByteChannel open(String spec) throws IOException { - LOG.debug("opening file {}", spec); - @SuppressWarnings("resource") // The caller is responsible for closing the channel. - FileInputStream inputStream = new FileInputStream(specToFile(spec)); - // Use this method for creating the channel (rather than new FileChannel) so that we get - // regular FileNotFoundException. Closing the underyling channel will close the inputStream. - return inputStream.getChannel(); - } - - @Override - public WritableByteChannel create(String spec, String mimeType) - throws IOException { - LOG.debug("creating file {}", spec); - File file = specToFile(spec); - if (file.getAbsoluteFile().getParentFile() != null - && !file.getAbsoluteFile().getParentFile().exists() - && !file.getAbsoluteFile().getParentFile().mkdirs() - && !file.getAbsoluteFile().getParentFile().exists()) { - throw new IOException("Unable to create parent directories for '" + spec + "'"); - } - return Channels.newChannel( - new BufferedOutputStream(new FileOutputStream(file))); - } - - @Override - public long getSizeBytes(String spec) throws IOException { - try { - return Files.size(specToFile(spec).toPath()); - } catch (NoSuchFileException e) { - throw new FileNotFoundException(e.getReason()); - } - } - - @Override - public boolean isReadSeekEfficient(String spec) throws IOException { - return true; - } - - @Override - public String resolve(String path, String other) throws IOException { - return toPath(path).resolve(other).toString(); - } - - @Override - public Path toPath(String path) { - return specToFile(path).toPath(); - } - - @Override - public void copy(Iterable srcFilenames, Iterable destFilenames) throws - IOException { - List srcList = Lists.newArrayList(srcFilenames); - List destList = Lists.newArrayList(destFilenames); - checkArgument( - srcList.size() == destList.size(), - "Number of source files %s must equal number of destination files %s", - srcList.size(), - destList.size()); - int numFiles = srcList.size(); - for (int i = 0; i < numFiles; i++) { - String src = srcList.get(i); - String dst = destList.get(i); - LOG.info("Copying {} to {}", src, dst); - try { - // Copy the source file, replacing the existing destination. - // Paths.get(x) will not work on Windows OSes cause of the ":" after the drive letter. - Files.copy( - new File(src).toPath(), - new File(dst).toPath(), - StandardCopyOption.REPLACE_EXISTING); - } catch (NoSuchFileException e) { - LOG.info("{} does not exist.", src); - // Suppress exception if file does not exist. - } - } - } - - @Override - public void remove(Collection filesOrDirs) throws IOException { - for (String fileOrDir : filesOrDirs) { - LOG.debug("Removing file {}", fileOrDir); - removeOne(fileOrDir); - } - } - - private void removeOne(String fileOrDir) throws IOException { - // Delete the file if it exists. - boolean exists = Files.deleteIfExists(Paths.get(fileOrDir)); - if (!exists) { - LOG.debug("Tried to delete {}, but it did not exist", fileOrDir); - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactoryRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactoryRegistrar.java deleted file mode 100644 index acc0222832f2..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactoryRegistrar.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import com.google.auto.service.AutoService; -import org.apache.beam.sdk.options.PipelineOptions; - -/** - * {@link AutoService} registrar for the {@link FileIOChannelFactory}. - */ -@AutoService(IOChannelFactoryRegistrar.class) -public class FileIOChannelFactoryRegistrar implements IOChannelFactoryRegistrar { - - @Override - public IOChannelFactory fromOptions(PipelineOptions options) { - return FileIOChannelFactory.fromOptions(options); - } - - @Override - public String getScheme() { - return "file"; - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java deleted file mode 100644 index 3a3af170fbca..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; -import java.nio.file.Path; -import java.util.Collection; - -/** - * Defines a factory for working with read and write channels. - * - *

Channels provide an abstract API for IO operations. - * - *

See Java NIO Channels - * - * @deprecated This is under redesign, see: https://issues.apache.org/jira/browse/BEAM-59. - */ -@Deprecated -public interface IOChannelFactory { - - /** - * Matches a specification, which may contain globs, against available - * resources. - * - *

Glob handling is dependent on the implementation. Implementations should - * all support globs in the final component of a path (eg /foo/bar/*.txt), - * however they are not required to support globs in the directory paths. - * - *

The list of resources returned are required to exist and not represent abstract - * resources such as symlinks and directories. - */ - Collection match(String spec) throws IOException; - - /** - * Returns a read channel for the given specification. - * - *

The specification is not expanded; it is used verbatim. - * - *

If seeking is supported, then this returns a - * {@link java.nio.channels.SeekableByteChannel}. - */ - ReadableByteChannel open(String spec) throws IOException; - - /** - * Returns a write channel for the given specification. - * - *

The specification is not expanded; is it used verbatim. - */ - WritableByteChannel create(String spec, String mimeType) throws IOException; - - /** - * Returns the size in bytes for the given specification. - * - *

The specification is not expanded; it is used verbatim. - * - *

{@link FileNotFoundException} will be thrown if the resource does not exist. - */ - long getSizeBytes(String spec) throws IOException; - - /** - * Returns {@code true} if the channel created when invoking method {@link #open} for the given - * file specification is guaranteed to be of type {@link java.nio.channels.SeekableByteChannel - * SeekableByteChannel} and if seeking into positions of the channel is recommended. Returns - * {@code false} if the channel returned is not a {@code SeekableByteChannel}. May return - * {@code false} even if the channel returned is a {@code SeekableByteChannel}, if seeking is not - * efficient for the given file specification. - * - *

Only efficiently seekable files can be split into offset ranges. - * - *

The specification is not expanded; it is used verbatim. - */ - boolean isReadSeekEfficient(String spec) throws IOException; - - /** - * Resolve the given {@code other} against the {@code path}. - * - *

If the {@code other} parameter is an absolute path then this method trivially returns - * other. If {@code other} is an empty path then this method trivially returns the given - * {@code path}. Otherwise this method considers the given {@code path} to be a directory and - * resolves the {@code other} path against this path. In the simplest case, the {@code other} - * path does not have a root component, in which case this method joins the {@code other} path - * to the given {@code path} and returns a resulting path that ends with the {@code other} path. - * Where the {@code other} path has a root component then resolution is highly implementation - * dependent and therefore unspecified. - */ - String resolve(String path, String other) throws IOException; - - /** Converts the given string to a {@link Path}. */ - Path toPath(String path); - - /** - * Copies a collection of files from one location to another. - * - *

The number of source filenames must equal the number of destination filenames. - * - * @param srcFilenames the source filenames. - * @param destFilenames the destination filenames. - */ - void copy(Iterable srcFilenames, Iterable destFilenames) throws IOException; - - /** - * Removes a collection of files or directories. - * - *

Directories are required to be empty. Non-empty directories will not be deleted, - * and this method may return silently or throw an exception. - */ - void remove(Collection filesOrDirs) throws IOException; -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactoryRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactoryRegistrar.java deleted file mode 100644 index 7776b130b771..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactoryRegistrar.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import com.google.auto.service.AutoService; -import java.util.ServiceLoader; -import org.apache.beam.sdk.options.PipelineOptions; - -/** - * A registrar that creates {@link IOChannelFactory} instances from {@link PipelineOptions}. - * - *

{@link IOChannelFactory} creators have the ability to provide a registrar by creating - * a {@link ServiceLoader} entry and a concrete implementation of this interface. - * - *

It is optional but recommended to use one of the many build time tools such as - * {@link AutoService} to generate the necessary META-INF files automatically. - */ -public interface IOChannelFactoryRegistrar { - /** - * Create a {@link IOChannelFactory} from the given {@link PipelineOptions}. - */ - IOChannelFactory fromOptions(PipelineOptions options); - - /** - * Get the URI scheme which defines the namespace of the IOChannelFactoryRegistrar. - * - *

The scheme is required to be unique among all - * {@link IOChannelFactoryRegistrar IOChannelFactoryRegistrars}. - * - * @see RFC 2396 - */ - String getScheme(); -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java index 33913f85b9c8..b658983eea87 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java @@ -17,264 +17,11 @@ */ package org.apache.beam.sdk.util; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.Lists; -import com.google.common.collect.Multimap; -import com.google.common.collect.Ordering; -import com.google.common.collect.Sets; -import com.google.common.collect.TreeMultimap; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Map.Entry; -import java.util.ServiceLoader; -import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import javax.annotation.Nonnull; -import org.apache.beam.sdk.io.DefaultFilenamePolicy; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.common.ReflectHelpers; -/** - * Provides utilities for creating read and write channels. - */ +/** Do not use, being removed. */ +@Deprecated public class IOChannelUtils { - // TODO: add registration mechanism for adding new schemas. - private static final Map FACTORY_MAP = - Collections.synchronizedMap(new HashMap()); - - private static final ClassLoader CLASS_LOADER = ReflectHelpers.findClassLoader(); - - /** - * Associates a scheme with an {@link IOChannelFactory}. - * - *

The given factory is used to construct read and write channels when - * a URI is provided with the given scheme. - * - *

For example, when reading from "gs://bucket/path", the scheme "gs" is - * used to lookup the appropriate factory. - * - *

{@link PipelineOptions} are required to provide dependencies and - * pipeline level configuration to the individual {@link IOChannelFactory IOChannelFactories}. - * - * @throws IllegalStateException if multiple {@link IOChannelFactory IOChannelFactories} - * for the same scheme are detected. - */ - @VisibleForTesting - public static void setIOFactoryInternal( - String scheme, - IOChannelFactory factory, - boolean override) { - if (!override && FACTORY_MAP.containsKey(scheme)) { - throw new IllegalStateException(String.format( - "Failed to register IOChannelFactory: %s. " - + "Scheme: [%s] is already registered with %s, and override is not allowed.", - FACTORY_MAP.get(scheme).getClass(), - scheme, - factory.getClass())); - } - FACTORY_MAP.put(scheme, factory); - } - - /** - * Deregisters the scheme and the associated {@link IOChannelFactory}. - */ - @VisibleForTesting - static void deregisterScheme(String scheme) { - FACTORY_MAP.remove(scheme); - } - - /** - * Registers all {@link IOChannelFactory IOChannelFactories} from {@link ServiceLoader}. - * - *

{@link PipelineOptions} are required to provide dependencies and - * pipeline level configuration to the individual {@link IOChannelFactory IOChannelFactories}. - * - *

Multiple {@link IOChannelFactory IOChannelFactories} for the same scheme are not allowed. - * - * @throws IllegalStateException if multiple {@link IOChannelFactory IOChannelFactories} - * for the same scheme are detected. - */ - public static void registerIOFactories(PipelineOptions options) { - registerIOFactoriesInternal(options, false /* override */); - } - - /** - * Registers all {@link IOChannelFactory IOChannelFactories} from {@link ServiceLoader}. - * - *

This requires {@link PipelineOptions} to provide, e.g., credentials for GCS. - * - *

Override existing schemes is allowed. - * - * @deprecated This is currently to provide different configurations for tests and - * is still public for IOChannelFactory redesign purposes. - */ @Deprecated - @VisibleForTesting - public static void registerIOFactoriesAllowOverride(PipelineOptions options) { - registerIOFactoriesInternal(options, true /* override */); - } - - private static void registerIOFactoriesInternal( - PipelineOptions options, boolean override) { - Set registrars = - Sets.newTreeSet(ReflectHelpers.ObjectsClassComparator.INSTANCE); - registrars.addAll(Lists.newArrayList( - ServiceLoader.load(IOChannelFactoryRegistrar.class, CLASS_LOADER))); - - checkDuplicateScheme(registrars); - - for (IOChannelFactoryRegistrar registrar : registrars) { - setIOFactoryInternal( - registrar.getScheme(), - registrar.fromOptions(options), - override); - } - } - - @VisibleForTesting - static void checkDuplicateScheme(Set registrars) { - Multimap registrarsBySchemes = - TreeMultimap.create(Ordering.natural(), Ordering.arbitrary()); - - for (IOChannelFactoryRegistrar registrar : registrars) { - registrarsBySchemes.put(registrar.getScheme(), registrar); - } - for (Entry> entry - : registrarsBySchemes.asMap().entrySet()) { - if (entry.getValue().size() > 1) { - String conflictingRegistrars = Joiner.on(", ").join( - FluentIterable.from(entry.getValue()) - .transform(new Function() { - @Override - public String apply(@Nonnull IOChannelFactoryRegistrar input) { - return input.getClass().getName(); - }}) - .toSortedList(Ordering.natural())); - throw new IllegalStateException(String.format( - "Scheme: [%s] has conflicting registrars: [%s]", - entry.getKey(), - conflictingRegistrars)); - } - } - } - - /** - * Creates a write channel for the given filename. - */ - public static WritableByteChannel create(String filename, String mimeType) - throws IOException { - return getFactory(filename).create(filename, mimeType); - } - - /** - * Creates a read channel for the given filename. - */ - public static ReadableByteChannel open(String filename) - throws IOException { - return getFactory(filename).open(filename); - } - - /** - * Creates a write channel for the given file components. - * - *

If numShards is specified, then a ShardingWritableByteChannel is - * returned. - * - *

Shard numbers are 0 based, meaning they start with 0 and end at the - * number of shards - 1. - */ - public static WritableByteChannel create(String prefix, String shardTemplate, - String suffix, int numShards, String mimeType) throws IOException { - if (numShards == 1) { - return create(DefaultFilenamePolicy.constructName(prefix, shardTemplate, suffix, 0, 1), - mimeType); - } - - // It is the callers responsibility to close this channel. - @SuppressWarnings("resource") - ShardingWritableByteChannel shardingChannel = - new ShardingWritableByteChannel(); - - Set outputNames = new HashSet<>(); - for (int i = 0; i < numShards; i++) { - String outputName = - DefaultFilenamePolicy.constructName(prefix, shardTemplate, suffix, i, numShards); - if (!outputNames.add(outputName)) { - throw new IllegalArgumentException( - "Shard name collision detected for: " + outputName); - } - WritableByteChannel channel = create(outputName, mimeType); - shardingChannel.addChannel(channel); - } - - return shardingChannel; - } - - /** - * Returns the size in bytes for the given specification. - * - *

The specification is not expanded; it is used verbatim. - * - *

{@link FileNotFoundException} will be thrown if the resource does not exist. - */ - public static long getSizeBytes(String spec) throws IOException { - return getFactory(spec).getSizeBytes(spec); - } - - private static final Pattern URI_SCHEME_PATTERN = Pattern.compile( - "(?[a-zA-Z][-a-zA-Z0-9+.]*)://.*"); - - /** - * Returns the IOChannelFactory associated with an input specification. - */ - public static IOChannelFactory getFactory(String spec) throws IOException { - // The spec is almost, but not quite, a URI. In particular, - // the reserved characters '[', ']', and '?' have meanings that differ - // from their use in the URI spec. ('*' is not reserved). - // Here, we just need the scheme, which is so circumscribed as to be - // very easy to extract with a regex. - Matcher matcher = URI_SCHEME_PATTERN.matcher(spec); - - if (!matcher.matches()) { - return FileIOChannelFactory.fromOptions(null); - } - - String scheme = matcher.group("scheme"); - IOChannelFactory ioFactory = FACTORY_MAP.get(scheme); - if (ioFactory != null) { - return ioFactory; - } - - throw new IOException("Unable to find handler for " + spec); - } - - /** - * Resolve multiple {@code others} against the {@code path} sequentially. - * - *

Empty paths in {@code others} are ignored. If {@code others} contains one or more - * absolute paths, then this method returns a path that starts with the last absolute path - * in {@code others} joined with the remaining paths. Resolution of paths is highly - * implementation dependent and therefore unspecified. - */ - public static String resolve(String path, String... others) throws IOException { - IOChannelFactory ioFactory = getFactory(path); - String fullPath = path; - - for (String other : others) { - fullPath = ioFactory.resolve(fullPath, other); - } - - return fullPath; - } + public static void registerIOFactoriesAllowOverride(PipelineOptions options) {} } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryRegistrarTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryRegistrarTest.java deleted file mode 100644 index f8f53e7a51fc..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryRegistrarTest.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static org.junit.Assert.fail; - -import com.google.common.collect.Lists; -import java.util.ServiceLoader; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Tests for {@link FileIOChannelFactoryRegistrar}. - */ -@RunWith(JUnit4.class) -public class FileIOChannelFactoryRegistrarTest { - - @Test - public void testServiceLoader() { - for (IOChannelFactoryRegistrar registrar - : Lists.newArrayList(ServiceLoader.load(IOChannelFactoryRegistrar.class).iterator())) { - if (registrar instanceof FileIOChannelFactoryRegistrar) { - return; - } - } - fail("Expected to find " + FileIOChannelFactoryRegistrar.class); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java deleted file mode 100644 index 60626195ebad..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java +++ /dev/null @@ -1,242 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; - -import com.google.common.collect.ImmutableList; -import com.google.common.io.Files; -import com.google.common.io.LineReader; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.Reader; -import java.io.Writer; -import java.nio.channels.Channels; -import java.nio.charset.StandardCharsets; -import java.nio.file.Path; -import java.util.List; -import org.hamcrest.Matchers; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for {@link FileIOChannelFactory}. */ -@RunWith(JUnit4.class) -public class FileIOChannelFactoryTest { - @Rule public ExpectedException thrown = ExpectedException.none(); - @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private FileIOChannelFactory factory = FileIOChannelFactory.fromOptions(null); - - private void testCreate(Path path) throws Exception { - String expected = "my test string"; - // First with the path string - try (Writer writer = Channels.newWriter( - factory.create(path.toString(), MimeTypes.TEXT), StandardCharsets.UTF_8.name())) { - writer.write(expected); - } - assertThat( - Files.readLines(path.toFile(), StandardCharsets.UTF_8), - containsInAnyOrder(expected)); - - // Delete the file before trying as URI - assertTrue("Unable to delete file " + path, path.toFile().delete()); - - // Second with the path URI - try (Writer writer = Channels.newWriter( - factory.create(path.toUri().toString(), MimeTypes.TEXT), StandardCharsets.UTF_8.name())) { - writer.write(expected); - } - assertThat( - Files.readLines(path.toFile(), StandardCharsets.UTF_8), - containsInAnyOrder(expected)); - } - - @Test - public void testCreateWithExistingFile() throws Exception { - File existingFile = temporaryFolder.newFile(); - testCreate(existingFile.toPath()); - } - - @Test - public void testCreateWithinExistingDirectory() throws Exception { - testCreate(temporaryFolder.getRoot().toPath().resolve("file.txt")); - } - - @Test - public void testCreateWithNonExistentSubDirectory() throws Exception { - testCreate(temporaryFolder.getRoot().toPath().resolve("non-existent-dir").resolve("file.txt")); - } - - @Test - public void testReadWithExistingFile() throws Exception { - String expected = "my test string"; - File existingFile = temporaryFolder.newFile(); - Files.write(expected, existingFile, StandardCharsets.UTF_8); - String data; - try (Reader reader = - Channels.newReader(factory.open(existingFile.getPath()), StandardCharsets.UTF_8.name())) { - data = new LineReader(reader).readLine(); - } - assertEquals(expected, data); - } - - @Test - public void testReadNonExistentFile() throws Exception { - thrown.expect(FileNotFoundException.class); - factory - .open( - temporaryFolder - .getRoot() - .toPath() - .resolve("non-existent-file.txt") - .toString()) - .close(); - } - - @Test - public void testIsReadSeekEfficient() throws Exception { - assertTrue(factory.isReadSeekEfficient("somePath")); - } - - @Test - public void testMatchExact() throws Exception { - List expected = ImmutableList.of(temporaryFolder.newFile("a").toString()); - temporaryFolder.newFile("aa"); - temporaryFolder.newFile("ab"); - - assertThat(factory.match(temporaryFolder.getRoot().toPath().resolve("a").toString()), - containsInAnyOrder(expected.toArray(new String[expected.size()]))); - } - - @Test - public void testMatchPatternNone() throws Exception { - List expected = ImmutableList.of(); - temporaryFolder.newFile("a"); - temporaryFolder.newFile("aa"); - temporaryFolder.newFile("ab"); - - // Windows doesn't like resolving paths with * in them, so the * is appended after resolve. - assertThat(factory.match(factory.resolve(temporaryFolder.getRoot().getPath(), "b") + "*"), - containsInAnyOrder(expected.toArray(new String[expected.size()]))); - } - - @Test - public void testMatchForNonExistentFile() throws Exception { - List expected = ImmutableList.of(); - temporaryFolder.newFile("aa"); - - assertThat(factory.match(factory.resolve(temporaryFolder.getRoot().getPath(), "a")), - containsInAnyOrder(expected.toArray(new String[expected.size()]))); - } - - @Test - public void testMatchMultipleWithoutSubdirectoryExpansion() throws Exception { - File unmatchedSubDir = temporaryFolder.newFolder("aaa"); - File unmatchedSubDirFile = File.createTempFile("sub-dir-file", "", unmatchedSubDir); - unmatchedSubDirFile.deleteOnExit(); - List expected = ImmutableList.of(temporaryFolder.newFile("a").toString(), - temporaryFolder.newFile("aa").toString(), temporaryFolder.newFile("ab").toString()); - temporaryFolder.newFile("ba"); - temporaryFolder.newFile("bb"); - - // Windows doesn't like resolving paths with * in them, so the * is appended after resolve. - assertThat(factory.match(factory.resolve(temporaryFolder.getRoot().getPath(), "a") + "*"), - containsInAnyOrder(expected.toArray(new String[expected.size()]))); - } - - @Test - public void testMatchMultipleWithSubdirectoryExpansion() throws Exception { - File matchedSubDir = temporaryFolder.newFolder("a"); - File matchedSubDirFile = File.createTempFile("sub-dir-file", "", matchedSubDir); - matchedSubDirFile.deleteOnExit(); - File unmatchedSubDir = temporaryFolder.newFolder("b"); - File unmatchedSubDirFile = File.createTempFile("sub-dir-file", "", unmatchedSubDir); - unmatchedSubDirFile.deleteOnExit(); - - List expected = ImmutableList.of(matchedSubDirFile.toString(), - temporaryFolder.newFile("aa").toString(), temporaryFolder.newFile("ab").toString()); - temporaryFolder.newFile("ba"); - temporaryFolder.newFile("bb"); - - // Windows doesn't like resolving paths with * in them, so the ** is appended after resolve. - assertThat(factory.match(factory.resolve(temporaryFolder.getRoot().getPath(), "a") + "**"), - Matchers.hasItems(expected.toArray(new String[expected.size()]))); - } - - @Test - public void testMatchWithDirectoryFiltersOutDirectory() throws Exception { - List expected = ImmutableList.of(temporaryFolder.newFile("a").toString()); - temporaryFolder.newFolder("a_dir_that_should_not_be_matched"); - - // Windows doesn't like resolving paths with * in them, so the * is appended after resolve. - assertThat(factory.match(factory.resolve(temporaryFolder.getRoot().getPath(), "a") + "*"), - containsInAnyOrder(expected.toArray(new String[expected.size()]))); - } - - @Test - public void testMatchWithoutParentDirectory() throws Exception { - String pattern = factory.resolve( - factory.resolve(temporaryFolder.getRoot().getPath(), "non_existing_dir"), - "*"); - assertTrue(factory.match(pattern).isEmpty()); - } - - @Test - public void testResolve() throws Exception { - Path rootPath = temporaryFolder.getRoot().toPath(); - String rootString = rootPath.toString(); - - String expected = rootPath.resolve("aa").toString(); - assertEquals(expected, factory.resolve(rootString, "aa")); - assertEquals(expected, factory.resolve("file:" + rootString, "aa")); - assertEquals(expected, factory.resolve("file://" + rootString, "aa")); - } - - @Test - public void testResolveOtherIsFullPath() throws Exception { - String expected = temporaryFolder.getRoot().getPath(); - assertEquals(expected, factory.resolve(expected, expected)); - } - - @Test - public void testResolveOtherIsEmptyPath() throws Exception { - String expected = temporaryFolder.getRoot().getPath(); - assertEquals(expected, factory.resolve(expected, "")); - } - - @Test - public void testGetSizeBytes() throws Exception { - String data = "TestData!!!"; - File file = temporaryFolder.newFile(); - Files.write(data, file, StandardCharsets.UTF_8); - assertEquals(data.length(), factory.getSizeBytes(file.getPath())); - } - - @Test - public void testGetSizeBytesForNonExistentFile() throws Exception { - thrown.expect(FileNotFoundException.class); - factory.getSizeBytes( - factory.resolve(temporaryFolder.getRoot().getPath(), "non-existent-file")); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java deleted file mode 100644 index ea4ae87ef075..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static org.hamcrest.Matchers.instanceOf; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; - -import com.google.common.collect.Sets; -import com.google.common.io.Files; -import java.io.File; -import java.nio.charset.StandardCharsets; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Tests for IOChannelUtils. - */ -@RunWith(JUnit4.class) -public class IOChannelUtilsTest { - @Rule - public TemporaryFolder tmpFolder = new TemporaryFolder(); - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Test(expected = IllegalArgumentException.class) - public void testShardNameCollision() throws Exception { - File outFolder = tmpFolder.newFolder(); - String filename = outFolder.toPath().resolve("output").toString(); - - IOChannelUtils.create(filename, "", "", 2, "text").close(); - fail("IOChannelUtils.create expected to fail due " - + "to filename collision"); - } - - @Test - public void testHandlerNoScheme() throws Exception { - String pathToTempFolder = tmpFolder.getRoot().getAbsolutePath(); - assertThat(IOChannelUtils.getFactory(pathToTempFolder), instanceOf(FileIOChannelFactory.class)); - } - - @Test - public void testGetSizeBytes() throws Exception { - String data = "TestData"; - File file = tmpFolder.newFile(); - Files.write(data, file, StandardCharsets.UTF_8); - assertEquals(data.length(), IOChannelUtils.getSizeBytes(file.getPath())); - } - - @Test - public void testResolveSinglePath() throws Exception { - String expected = tmpFolder.getRoot().toPath().resolve("aa").toString(); - assertEquals(expected, IOChannelUtils.resolve(tmpFolder.getRoot().toString(), "aa")); - } - - @Test - public void testResolveMultiplePaths() throws Exception { - String expected = - tmpFolder.getRoot().toPath().resolve("aa").resolve("bb").resolve("cc").toString(); - assertEquals(expected, - IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "aa", "bb", "cc")); - } - - @Test - public void testRegisterIOFactoriesAllowOverride() throws Exception { - IOChannelUtils.registerIOFactoriesAllowOverride(PipelineOptionsFactory.create()); - IOChannelUtils.registerIOFactoriesAllowOverride(PipelineOptionsFactory.create()); - assertNotNull(IOChannelUtils.getFactory("gs")); - assertNotNull(IOChannelUtils.getFactory("file")); - } - - @Test - public void testRegisterIOFactories() throws Exception { - IOChannelUtils.deregisterScheme("gs"); - IOChannelUtils.deregisterScheme("file"); - - IOChannelUtils.registerIOFactories(PipelineOptionsFactory.create()); - assertNotNull(IOChannelUtils.getFactory("gs")); - assertNotNull(IOChannelUtils.getFactory("file")); - thrown.expect(RuntimeException.class); - thrown.expectMessage("Failed to register IOChannelFactory"); - thrown.expectMessage("override is not allowed"); - IOChannelUtils.registerIOFactories(PipelineOptionsFactory.create()); - } - - @Test - public void testCheckDuplicateScheme() throws Exception { - thrown.expect(RuntimeException.class); - thrown.expectMessage("Scheme: [file] has conflicting registrars"); - IOChannelUtils.checkDuplicateScheme( - Sets.newHashSet( - new FileIOChannelFactoryRegistrar(), - new FileIOChannelFactoryRegistrar())); - } -} diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java deleted file mode 100644 index 3a12620d0f2c..000000000000 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import java.io.IOException; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; -import java.nio.file.Path; -import java.util.Collection; -import java.util.LinkedList; -import java.util.List; -import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.gcsfs.GcsPath; - -/** - * Implements IOChannelFactory for GCS. - */ -public class GcsIOChannelFactory implements IOChannelFactory { - - /** - * Create a {@link GcsIOChannelFactory} with the given {@link PipelineOptions}. - */ - public static GcsIOChannelFactory fromOptions(PipelineOptions options) { - return new GcsIOChannelFactory(options.as(GcsOptions.class)); - } - - private final GcsOptions options; - - private GcsIOChannelFactory(GcsOptions options) { - this.options = options; - } - - @Override - public Collection match(String spec) throws IOException { - GcsPath path = GcsPath.fromUri(spec); - GcsUtil util = options.getGcsUtil(); - List matched = util.expand(path); - - List specs = new LinkedList<>(); - for (GcsPath match : matched) { - specs.add(match.toString()); - } - - return specs; - } - - @Override - public ReadableByteChannel open(String spec) throws IOException { - GcsPath path = GcsPath.fromUri(spec); - GcsUtil util = options.getGcsUtil(); - return util.open(path); - } - - @Override - public WritableByteChannel create(String spec, String mimeType) - throws IOException { - GcsPath path = GcsPath.fromUri(spec); - GcsUtil util = options.getGcsUtil(); - return util.create(path, mimeType); - } - - @Override - public long getSizeBytes(String spec) throws IOException { - GcsPath path = GcsPath.fromUri(spec); - GcsUtil util = options.getGcsUtil(); - return util.fileSize(path); - } - - @Override - public boolean isReadSeekEfficient(String spec) throws IOException { - // TODO It is incorrect to return true here for files with content encoding set to gzip. - return true; - } - - @Override - public String resolve(String path, String other) throws IOException { - return toPath(path).resolve(other).toString(); - } - - @Override - public Path toPath(String path) { - return GcsPath.fromUri(path); - } - - @Override - public void copy(Iterable srcFilenames, Iterable destFilenames) - throws IOException { - options.getGcsUtil().copy(srcFilenames, destFilenames); - } - - @Override - public void remove(Collection filesOrDirs) throws IOException { - options.getGcsUtil().remove(filesOrDirs); - } -} diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java deleted file mode 100644 index b4c457fcaeb7..000000000000 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrar.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import com.google.auto.service.AutoService; -import org.apache.beam.sdk.options.PipelineOptions; - -/** - * {@link AutoService} registrar for the {@link GcsIOChannelFactory}. - */ -@AutoService(IOChannelFactoryRegistrar.class) -public class GcsIOChannelFactoryRegistrar implements IOChannelFactoryRegistrar { - - @Override - public GcsIOChannelFactory fromOptions(PipelineOptions options) { - return GcsIOChannelFactory.fromOptions(options); - } - - @Override - public String getScheme() { - return "gs"; - } -} diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java deleted file mode 100644 index a29dd45456e4..000000000000 --- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryRegistrarTest.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static org.junit.Assert.fail; - -import com.google.common.collect.Lists; -import java.util.ServiceLoader; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Tests for {@link GcsIOChannelFactoryRegistrar}. - */ -@RunWith(JUnit4.class) -public class GcsIOChannelFactoryRegistrarTest { - - @Test - public void testServiceLoader() { - for (IOChannelFactoryRegistrar registrar - : Lists.newArrayList(ServiceLoader.load(IOChannelFactoryRegistrar.class).iterator())) { - if (registrar instanceof GcsIOChannelFactoryRegistrar) { - return; - } - } - fail("Expected to find " + GcsIOChannelFactoryRegistrar.class); - } -} diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java deleted file mode 100644 index f53490add74e..000000000000 --- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import static org.junit.Assert.assertEquals; - -import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for {@link GcsIOChannelFactoryTest}. */ -@RunWith(JUnit4.class) -public class GcsIOChannelFactoryTest { - private GcsIOChannelFactory factory; - - @Before - public void setUp() { - factory = GcsIOChannelFactory.fromOptions(PipelineOptionsFactory.as(GcsOptions.class)); - } - - @Test - public void testResolve() throws Exception { - assertEquals("gs://bucket/object", factory.resolve("gs://bucket", "object")); - } -} From 1d9089c968535090ba77977ddecc6cdfb936655f Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Thu, 4 May 2017 12:48:55 -0700 Subject: [PATCH 043/387] Move thrown.expect to correct place --- .../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 630893147f6f..dbeab04c9731 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 @@ -287,11 +287,11 @@ public void testWritingFailsTableDoesNotExist() throws Exception { PCollection>> emptyInput = p.apply(Create.empty(HBaseIO.WRITE_CODER)); + emptyInput.apply("write", HBaseIO.write().withConfiguration(conf).withTableId(table)); + // Exception will be thrown by write.validate() when write is applied. thrown.expect(IllegalArgumentException.class); thrown.expectMessage(String.format("Table %s does not exist", table)); - - emptyInput.apply("write", HBaseIO.write().withConfiguration(conf).withTableId(table)); p.run(); } From 26073817bdfcd64c26dadcb9ed3940a7fc987c59 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 17 Apr 2017 13:21:14 -0700 Subject: [PATCH 044/387] Note when checkpoints may be committed. --- .../apache/beam/sdk/io/UnboundedSource.java | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java index af6a8ccd9e56..b99d5eefaa29 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java @@ -83,9 +83,9 @@ public abstract UnboundedReader createReader( /** * Returns whether this source requires explicit deduping. * - *

This is needed if the underlying data source can return the same record multiple times, - * such a queuing system with a pull-ack model. Sources where the records read are uniquely - * identified by the persisted state in the CheckpointMark do not need this. + *

This is needed if the underlying data source can return the same record multiple times, such + * a queuing system with a pull-ack model. Sources where the records read are uniquely identified + * by the persisted state in the CheckpointMark do not need this. * *

Generally, if {@link CheckpointMark#finalizeCheckpoint()} is overridden, this method should * return true. Checkpoint finalization is best-effort, and readers can be resumed from a @@ -233,20 +233,25 @@ public byte[] getCurrentRecordId() throws NoSuchElementException { * } * } * - *

All elements read up until this method is called will be processed together as a bundle. - * (An element is considered 'read' if it could be returned by a call to {@link #getCurrent}.) - * Once the result of processing those elements and the returned checkpoint have been durably + *

All elements read between the last time this method was called (or since this reader was + * created, if this method has not been called on this reader) until this method is called will + * be processed together as a bundle. (An element is considered 'read' if it could be returned + * by a call to {@link #getCurrent}.) + * + *

Once the result of processing those elements and the returned checkpoint have been durably * committed, {@link CheckpointMark#finalizeCheckpoint} will be called at most once at some * later point on the returned {@link CheckpointMark} object. Checkpoint finalization is * best-effort, and checkpoints may not be finalized. If duplicate elements may be produced if * checkpoints are not finalized in a timely manner, {@link UnboundedSource#requiresDeduping()} * should be overridden to return true, and {@link UnboundedReader#getCurrentRecordId()} should - * be overriden to return unique record IDs. + * be overridden to return unique record IDs. + * + *

A checkpoint will be committed to durable storage only if all all previous checkpoints + * produced by the same reader have also been committed. * *

The returned object should not be modified. * - *

May be called after {@link #advance} or {@link #start} has returned false, but not before - * {@link #start} has been called. + *

May not be called before {@link #start} has been called. */ public abstract CheckpointMark getCheckpointMark(); From 513ef1145aa6c563625ab4154296d5aa4fc6c17c Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 4 May 2017 15:01:24 -0700 Subject: [PATCH 045/387] Activate WindowedWordCountIT on Apex runner --- examples/java/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index d673da27fe29..65625df297b3 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -303,6 +303,7 @@ WordCountIT.java + WindowedWordCountIT.java all 4 From d59d9b74ecfd8bd936c768addab25e714a481ed0 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 1 May 2017 18:11:18 -0700 Subject: [PATCH 046/387] Split StartBundleContext and FinishBundleContext These methods have different restrictions on their capabilities. Remove Context.output(), and add a more precise FinishBundleContext.output(Element, Timestamp, Window) --- .../runners/core/BaseExecutionContext.java | 9 - .../beam/runners/core/DoFnAdapters.java | 111 +++++++++--- .../beam/runners/core/ExecutionContext.java | 10 +- ...oundedSplittableProcessElementInvoker.java | 13 +- .../beam/runners/core/SimpleDoFnRunner.java | 161 +++++++++++++++--- .../beam/runners/core/SplittableParDo.java | 35 ++-- .../runners/core/SimpleDoFnRunnerTest.java | 4 +- .../runners/core/SplittableParDoTest.java | 4 +- runners/google-cloud-dataflow-java/pom.xml | 2 +- .../runners/dataflow/BatchViewOverrides.java | 2 +- .../org/apache/beam/sdk/io/WriteFiles.java | 7 +- .../beam/sdk/options/PipelineOptions.java | 3 +- .../apache/beam/sdk/transforms/Combine.java | 5 +- .../org/apache/beam/sdk/transforms/DoFn.java | 79 ++++++--- .../beam/sdk/transforms/DoFnTester.java | 62 ++++--- .../beam/sdk/transforms/GroupIntoBatches.java | 2 +- .../org/apache/beam/sdk/transforms/ParDo.java | 5 +- .../reflect/ByteBuddyDoFnInvokerFactory.java | 19 ++- .../sdk/transforms/reflect/DoFnInvoker.java | 19 ++- .../sdk/transforms/reflect/DoFnSignature.java | 42 +++-- .../transforms/reflect/DoFnSignatures.java | 58 +++++-- .../sdk/transforms/windowing/PaneInfo.java | 3 +- .../apache/beam/sdk/metrics/MetricsTest.java | 4 +- .../beam/sdk/transforms/DoFnTesterTest.java | 8 +- .../sdk/transforms/ParDoLifecycleTest.java | 16 +- .../apache/beam/sdk/transforms/ParDoTest.java | 75 ++------ .../sdk/transforms/SplittableDoFnTest.java | 4 +- .../transforms/reflect/DoFnInvokersTest.java | 22 ++- .../reflect/DoFnSignaturesTest.java | 15 +- .../control/ProcessBundleHandlerTest.java | 17 +- .../sdk/io/elasticsearch/ElasticsearchIO.java | 12 +- .../sdk/io/gcp/bigquery/StreamingWriteFn.java | 4 +- .../sdk/io/gcp/bigquery/TagWithUniqueIds.java | 2 +- .../io/gcp/bigquery/WriteBundlesToFiles.java | 17 +- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 4 +- .../sdk/io/gcp/datastore/DatastoreV1.java | 10 +- .../beam/sdk/io/gcp/pubsub/PubsubIO.java | 4 +- .../io/gcp/pubsub/PubsubUnboundedSink.java | 4 +- .../org/apache/beam/sdk/io/hdfs/Write.java | 7 +- .../org/apache/beam/sdk/io/hbase/HBaseIO.java | 7 +- .../org/apache/beam/sdk/io/jdbc/JdbcIO.java | 11 +- .../org/apache/beam/sdk/io/jms/JmsIO.java | 6 +- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 2 +- .../beam/sdk/io/mongodb/MongoDbGridFSIO.java | 4 +- .../apache/beam/sdk/io/mongodb/MongoDbIO.java | 10 +- 45 files changed, 603 insertions(+), 317 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 8d682563f692..23d61f89b1db 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 @@ -23,7 +23,6 @@ import java.util.LinkedHashMap; import java.util.Map; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.DoFn.Context; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; @@ -105,17 +104,9 @@ public Collection getAllStepContexts() { return Collections.unmodifiableCollection(cachedStepContexts.values()); } - /** - * Hook for subclasses to implement that will be called whenever - * {@link Context#output(Object)} is called. - */ @Override public void noteOutput(WindowedValue output) {} - /** - * Hook for subclasses to implement that will be called whenever - * {@link Context#output(TupleTag, Object)} is called. - */ @Override public void noteOutput(TupleTag tag, WindowedValue output) {} diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java index b2069e19d010..fd1772ad2761 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java @@ -18,11 +18,13 @@ package org.apache.beam.runners.core; import java.io.IOException; +import org.apache.beam.runners.core.OldDoFn.Context; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.Context; +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.display.DisplayData; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; @@ -81,12 +83,12 @@ public void setup() throws Exception { @Override public void startBundle(Context c) throws Exception { fn.prepareForProcessing(); - invoker.invokeStartBundle(new ContextAdapter<>(fn, c)); + invoker.invokeStartBundle(new StartBundleContextAdapter<>(fn, c)); } @Override public void finishBundle(Context c) throws Exception { - invoker.invokeFinishBundle(new ContextAdapter<>(fn, c)); + invoker.invokeFinishBundle(new FinishBundleContextAdapter<>(fn, c)); } @Override @@ -128,42 +130,86 @@ private static class WindowDoFnAdapter extends SimpleDoFnAdapte /** * Wraps an {@link OldDoFn.Context} as a {@link DoFnInvoker.ArgumentProvider} inside a {@link - * DoFn.StartBundle} or {@link DoFn.FinishBundle} method, which means the extra context is - * unavailable. + * DoFn.StartBundle} method, which means the extra context is unavailable. */ - private static class ContextAdapter extends DoFn.Context + private static class StartBundleContextAdapter + extends DoFn.StartBundleContext implements DoFnInvoker.ArgumentProvider { - private OldDoFn.Context context; - private ContextAdapter(DoFn fn, OldDoFn.Context context) { + private StartBundleContextAdapter(DoFn fn, Context context) { fn.super(); this.context = context; } - @Override public PipelineOptions getPipelineOptions() { return context.getPipelineOptions(); } @Override - public void output(OutputT output) { - context.output(output); + public BoundedWindow window() { + // The OldDoFn doesn't allow us to ask for these outside processElement, so this + // should be unreachable. + throw new UnsupportedOperationException( + "Can only get the window in processElement; elsewhere there is no defined window."); } @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - context.outputWithTimestamp(output, timestamp); + public StartBundleContext startBundleContext(DoFn doFn) { + return this; } @Override - public void output(TupleTag tag, T output) { - context.output(tag, output); + public FinishBundleContext finishBundleContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Can only get a FinishBundleContext in finishBundle"); } @Override - public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - context.outputWithTimestamp(tag, output, timestamp); + public ProcessContext processContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Can only get a ProcessContext in processElement"); + } + + @Override + public OnTimerContext onTimerContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Timers are not supported for OldDoFn"); + } + + @Override + public RestrictionTracker restrictionTracker() { + throw new UnsupportedOperationException("This is a non-splittable DoFn"); + } + + @Override + public State state(String stateId) { + throw new UnsupportedOperationException("State is not supported by this runner"); + } + + @Override + public Timer timer(String timerId) { + throw new UnsupportedOperationException("Timers are not supported by this runner"); + } + } + + /** + * Wraps an {@link OldDoFn.Context} as a {@link DoFnInvoker.ArgumentProvider} inside a {@link + * DoFn.FinishBundle} method, which means the extra context is unavailable. + */ + private static class FinishBundleContextAdapter + extends DoFn.FinishBundleContext + implements DoFnInvoker.ArgumentProvider { + + private OldDoFn.Context context; + + private FinishBundleContextAdapter(DoFn fn, Context context) { + fn.super(); + this.context = context; + } + + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); } @Override @@ -175,7 +221,13 @@ public BoundedWindow window() { } @Override - public Context context(DoFn doFn) { + public StartBundleContext startBundleContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Can only get a StartBundleContext in startBundle"); + } + + @Override + public FinishBundleContext finishBundleContext(DoFn doFn) { return this; } @@ -205,6 +257,20 @@ public State state(String stateId) { public Timer timer(String timerId) { throw new UnsupportedOperationException("Timers are not supported by this runner"); } + + @Override + public void output( + OutputT output, Instant timestamp, BoundedWindow window) { + // Not full fidelity conversion. This should be removed as soon as possible. + context.outputWithTimestamp(output, timestamp); + } + + @Override + public void output( + TupleTag tag, T output, Instant timestamp, BoundedWindow window) { + // Not full fidelity conversion. This should be removed as soon as possible. + context.outputWithTimestamp(tag, output, timestamp); + } } /** @@ -278,8 +344,13 @@ public BoundedWindow window() { } @Override - public Context context(DoFn doFn) { - return this; + public StartBundleContext startBundleContext(DoFn doFn) { + return null; + } + + @Override + public FinishBundleContext finishBundleContext(DoFn doFn) { + return null; } @Override 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 c0d01aeed3b9..d2fdaac50982 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 @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.Collection; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.DoFn.Context; +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; @@ -42,13 +42,13 @@ public interface ExecutionContext { /** * Hook for subclasses to implement that will be called whenever - * {@link Context#output(TupleTag, Object)} is called. + * {@link WindowedContext#output(TupleTag, Object)} is called. */ void noteOutput(WindowedValue output); /** * Hook for subclasses to implement that will be called whenever - * {@link Context#output(TupleTag, Object)} is called. + * {@link WindowedContext#output(TupleTag, Object)} is called. */ void noteOutput(TupleTag tag, WindowedValue output); @@ -69,14 +69,14 @@ public interface StepContext { /** * Hook for subclasses to implement that will be called whenever - * {@link org.apache.beam.sdk.transforms.DoFn.Context#output} + * {@link WindowedContext#output} * is called. */ void noteOutput(WindowedValue output); /** * Hook for subclasses to implement that will be called whenever - * {@link org.apache.beam.sdk.transforms.DoFn.Context#output} + * {@link WindowedContext#output} * is called. */ void noteOutput(TupleTag tag, WindowedValue output); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index 35d6eb7e2bd6..a51041ed9a99 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -28,6 +28,8 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.FinishBundleContext; +import org.apache.beam.sdk.transforms.DoFn.StartBundleContext; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -117,9 +119,16 @@ public BoundedWindow window() { } @Override - public DoFn.Context context(DoFn doFn) { + public StartBundleContext startBundleContext(DoFn doFn) { throw new IllegalStateException( - "Should not access context() from @" + "Should not access startBundleContext() from @" + + DoFn.ProcessElement.class.getSimpleName()); + } + + @Override + public FinishBundleContext finishBundleContext(DoFn doFn) { + throw new IllegalStateException( + "Should not access finishBundleContext() from @" + DoFn.ProcessElement.class.getSimpleName()); } 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 7f29a6fff52e..768378daceca 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 @@ -33,9 +33,10 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.Context; +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; @@ -137,9 +138,11 @@ public SimpleDoFnRunner( @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(context); + invoker.invokeStartBundle(startBundleContext); } catch (Throwable t) { // Exception in user code. throw wrapUserCodeException(t); @@ -200,15 +203,27 @@ 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(context); + invoker.invokeFinishBundle(finishBundleContext); } catch (Throwable t) { // Exception in user code. throw wrapUserCodeException(t); } } + private DoFnStartBundleContext createStartBundleContext( + DoFn fn, DoFnContext context) { + return new DoFnStartBundleContext<>(this.fn, this.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); @@ -228,8 +243,7 @@ private boolean isSystemDoFn() { * @param the type of the {@link DoFn} (main) input elements * @param the type of the {@link DoFn} (main) output elements */ - private static class DoFnContext extends DoFn.Context - implements DoFnInvoker.ArgumentProvider { + private static class DoFnContext { private static final int MAX_SIDE_OUTPUTS = 1000; final PipelineOptions options; @@ -255,7 +269,6 @@ public DoFnContext( List> additionalOutputTags, StepContext stepContext, WindowFn windowFn) { - fn.super(); this.options = options; this.fn = fn; this.sideInputReader = sideInputReader; @@ -274,7 +287,6 @@ public DoFnContext( ////////////////////////////////////////////////////////////////////////////// - @Override public PipelineOptions getPipelineOptions() { return options; } @@ -374,30 +386,95 @@ private void outputWindowedValue(TupleTag tag, WindowedValue windowedE stepContext.noteOutput(tag, windowedElem); } } + } + + + /** + * A concrete implementation of {@link DoFn.StartBundleContext}. + */ + private class DoFnStartBundleContext + extends DoFn.StartBundleContext + implements DoFnInvoker.ArgumentProvider { + private final DoFn fn; + private final DoFnContext context; + + private DoFnStartBundleContext(DoFn fn, DoFnContext context) { + fn.super(); + this.fn = fn; + this.context = context; + } - // Following implementations of output, outputWithTimestamp, and output - // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by - // ProcessContext's versions in DoFn.processElement. @Override - public void output(OutputT output) { - outputWindowedValue(output, null, null, PaneInfo.NO_FIRING); + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); } @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING); + public BoundedWindow window() { + throw new UnsupportedOperationException( + "Cannot access window outside of @ProcessElement and @OnTimer methods."); } @Override - public void output(TupleTag tag, T output) { - checkNotNull(tag, "TupleTag passed to output cannot be null"); - outputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING); + public StartBundleContext startBundleContext(DoFn doFn) { + return this; } @Override - public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - checkNotNull(tag, "TupleTag passed to outputWithTimestamp cannot be null"); - outputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING); + public FinishBundleContext finishBundleContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Cannot access FinishBundleContext outside of @FinishBundle method."); + } + + @Override + public ProcessContext processContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Cannot access ProcessContext outside of @ProcessElement method."); + } + + @Override + public OnTimerContext onTimerContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Cannot access OnTimerContext outside of @OnTimer methods."); + } + + @Override + public RestrictionTracker restrictionTracker() { + throw new UnsupportedOperationException( + "Cannot access RestrictionTracker outside of @ProcessElement method."); + } + + @Override + public State state(String stateId) { + throw new UnsupportedOperationException( + "Cannot access state outside of @ProcessElement and @OnTimer methods."); + } + + @Override + public Timer timer(String timerId) { + throw new UnsupportedOperationException( + "Cannot access timers outside of @ProcessElement and @OnTimer methods."); + } + } + + /** + * B + * A concrete implementation of {@link DoFn.FinishBundleContext}. + */ + private class DoFnFinishBundleContext + extends DoFn.FinishBundleContext + implements DoFnInvoker.ArgumentProvider { + private final DoFnContext context; + + private DoFnFinishBundleContext( + DoFn fn, DoFnContext context) { + fn.super(); + this.context = context; + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); } @Override @@ -407,14 +484,20 @@ public BoundedWindow window() { } @Override - public Context context(DoFn doFn) { + public StartBundleContext startBundleContext(DoFn doFn) { + throw new UnsupportedOperationException( + "Cannot access StartBundleContext outside of @StartBundle method."); + } + + @Override + public FinishBundleContext finishBundleContext(DoFn doFn) { return this; } @Override public ProcessContext processContext(DoFn doFn) { throw new UnsupportedOperationException( - "Cannot access ProcessContext outside of @Processelement method."); + "Cannot access ProcessContext outside of @ProcessElement method."); } @Override @@ -440,6 +523,17 @@ public Timer timer(String timerId) { throw new UnsupportedOperationException( "Cannot access timers outside of @ProcessElement and @OnTimer methods."); } + + @Override + public void output(OutputT output, Instant timestamp, BoundedWindow window) { + context.outputWindowedValue(WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); + } + + @Override + public void output(TupleTag tag, T output, Instant timestamp, BoundedWindow window) { + context.outputWindowedValue( + tag, WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); + } } /** @@ -590,8 +684,13 @@ public BoundedWindow window() { } @Override - public DoFn.Context context(DoFn doFn) { - return this; + public StartBundleContext startBundleContext(DoFn doFn) { + throw new UnsupportedOperationException("StartBundleContext parameters are not supported."); + } + + @Override + public FinishBundleContext finishBundleContext(DoFn doFn) { + throw new UnsupportedOperationException("FinishBundleContext parameters are not supported."); } @Override @@ -698,15 +797,21 @@ public BoundedWindow window() { } @Override - public TimeDomain timeDomain() { - return timeDomain; + public StartBundleContext startBundleContext(DoFn doFn) { + throw new UnsupportedOperationException("StartBundleContext parameters are not supported."); } @Override - public Context context(DoFn doFn) { - throw new UnsupportedOperationException("Context parameters are not supported."); + public FinishBundleContext finishBundleContext(DoFn doFn) { + throw new UnsupportedOperationException("FinishBundleContext parameters are not supported."); } + @Override + public TimeDomain timeDomain() { + return timeDomain; + } + + @Override public ProcessContext processContext(DoFn doFn) { throw new UnsupportedOperationException("ProcessContext parameters are not supported."); 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/SplittableParDo.java index ed065a67945a..7cf5e88a4c99 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/SplittableParDo.java @@ -440,13 +440,13 @@ public void tearDown() throws Exception { } @StartBundle - public void startBundle(Context c) throws Exception { - invoker.invokeStartBundle(wrapContext(c)); + public void startBundle(StartBundleContext c) throws Exception { + invoker.invokeStartBundle(wrapContextAsStartBundle(c)); } @FinishBundle - public void finishBundle(Context c) throws Exception { - invoker.invokeFinishBundle(wrapContext(c)); + public void finishBundle(FinishBundleContext c) throws Exception { + invoker.invokeFinishBundle(wrapContextAsFinishBundle(c)); } @ProcessElement @@ -522,31 +522,39 @@ public void processElement(final ProcessContext c) { stateNamespace, timerInternals.currentProcessingTime(), TimeDomain.PROCESSING_TIME)); } - private DoFn.Context wrapContext(final Context baseContext) { - return fn.new Context() { + private DoFn.StartBundleContext wrapContextAsStartBundle( + final StartBundleContext baseContext) { + return fn.new StartBundleContext() { @Override public PipelineOptions getPipelineOptions() { return baseContext.getPipelineOptions(); } - @Override - public void output(OutputT output) { - throwUnsupportedOutput(); + private void throwUnsupportedOutput() { + throw new UnsupportedOperationException( + String.format( + "Splittable DoFn can only output from @%s", + ProcessElement.class.getSimpleName())); } + }; + } + private DoFn.FinishBundleContext wrapContextAsFinishBundle( + final FinishBundleContext baseContext) { + return fn.new FinishBundleContext() { @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { + public void output(OutputT output, Instant timestamp, BoundedWindow window) { throwUnsupportedOutput(); } @Override - public void output(TupleTag tag, T output) { + public void output(TupleTag tag, T output, Instant timestamp, BoundedWindow window) { throwUnsupportedOutput(); } @Override - public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - throwUnsupportedOutput(); + public PipelineOptions getPipelineOptions() { + return baseContext.getPipelineOptions(); } private void throwUnsupportedOutput() { @@ -557,6 +565,7 @@ private void throwUnsupportedOutput() { } }; } + } /** Splits the restriction using the given {@link DoFn.SplitRestriction} method. */ 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 9b63bab94c1e..b8fc64ea7a55 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 @@ -348,12 +348,12 @@ static class ThrowingDoFn extends DoFn { private static final TimerSpec timer = TimerSpecs.timer(TimeDomain.EVENT_TIME); @StartBundle - public void startBundle(Context c) throws Exception { + public void startBundle() throws Exception { throw exceptionToThrow; } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle() throws Exception { throw exceptionToThrow; } 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/SplittableParDoTest.java index a67db6d82ef9..0bb25fcfa747 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/SplittableParDoTest.java @@ -578,13 +578,13 @@ public void tearDown() { } @StartBundle - public void startBundle(Context c) { + public void startBundle() { assertEquals(State.OUTSIDE_BUNDLE, state); state = State.INSIDE_BUNDLE; } @FinishBundle - public void finishBundle(Context c) { + public void finishBundle() { assertEquals(State.INSIDE_BUNDLE, state); state = State.OUTSIDE_BUNDLE; } diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index b0ae757b51aa..0367bd8a95a1 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -33,7 +33,7 @@ jar - beam-master-20170503 + beam-master-20170504 1 6 diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index 64fe49504b2b..b85b315da869 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -1021,7 +1021,7 @@ static class ToIsmRecordForGlobalWindowDoFn long indexInBundle; @StartBundle - public void startBundle(Context c) throws Exception { + public void startBundle() throws Exception { indexInBundle = 0; } 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 2a057e4fbf09..5b219027e64c 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 @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - import java.util.List; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; @@ -246,6 +245,7 @@ private class WriteBundles extends DoFn { // Writer that will write the records in this bundle. Lazily // initialized in processElement. private FileBasedWriter writer = null; + private BoundedWindow window = null; WriteBundles() { } @@ -263,6 +263,7 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except } else { writer.openUnwindowed(UUID.randomUUID().toString(), UNKNOWN_SHARDNUM, UNKNOWN_NUMSHARDS); } + this.window = window; LOG.debug("Done opening writer {} for operation {}", writer, writeOperation); } try { @@ -285,10 +286,10 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle(FinishBundleContext c) throws Exception { if (writer != null) { FileResult result = writer.close(); - c.output(result); + c.output(result, window.maxTimestamp(), window); // Reset state in case of reuse. writer = null; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java index d423bfadf03f..37081fde1a79 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java @@ -34,7 +34,6 @@ import org.apache.beam.sdk.options.ProxyInvocationHandler.Serializer; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.Context; import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; @@ -56,7 +55,7 @@ * from command-line arguments with {@link PipelineOptionsFactory#fromArgs(String[])}. * They can be converted to another type by invoking {@link PipelineOptions#as(Class)} and * can be accessed from within a {@link DoFn} by invoking - * {@link Context#getPipelineOptions()}. + * {@code getPipelineOptions()} on the input {@link DoFn.ProcessContext Context} object. * *

For example: *

{@code
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
index 0be8517da9dc..9d0e97ab16aa 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
@@ -1844,9 +1844,8 @@ public void populateDisplayData(DisplayData.Builder builder) {
           new DoFn, KV>() {
             transient int counter;
             @StartBundle
-            public void startBundle(Context c) {
-              counter = ThreadLocalRandom.current().nextInt(
-                  Integer.MAX_VALUE);
+            public void startBundle() {
+              counter = ThreadLocalRandom.current().nextInt(Integer.MAX_VALUE);
             }
 
             @ProcessElement
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index eab08f6b8793..f3d178e16f84 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -82,15 +82,62 @@
  * @param  the type of the (main) output elements
  */
 public abstract class DoFn implements Serializable, HasDisplayData {
+  /**
+   * Information accessible while within the {@link StartBundle} method.
+   */
+  public abstract class StartBundleContext {
+    /**
+     * Returns the {@code PipelineOptions} specified with the {@link
+     * org.apache.beam.sdk.runners.PipelineRunner} invoking this {@code DoFn}. The {@code
+     * PipelineOptions} will be the default running via {@link DoFnTester}.
+     */
+    public abstract PipelineOptions getPipelineOptions();
+  }
+
+  /**
+   * Information accessible while within the {@link FinishBundle} method.
+   */
+  public abstract class FinishBundleContext {
+    /**
+     * Returns the {@code PipelineOptions} specified with the {@link
+     * org.apache.beam.sdk.runners.PipelineRunner} invoking this {@code DoFn}. The {@code
+     * PipelineOptions} will be the default running via {@link DoFnTester}.
+     */
+    public abstract PipelineOptions getPipelineOptions();
+
+    /**
+     * Adds the given element to the main output {@code PCollection} at the given
+     * timestamp in the given window.
+     *
+     * 

Once passed to {@code output} the element should not be modified in + * any way. + * + *

Note: A splittable {@link DoFn} is not allowed to output from the + * {@link FinishBundle} method. + */ + public abstract void output(OutputT output, Instant timestamp, BoundedWindow window); - /** Information accessible to all methods in this {@code DoFn}. */ - public abstract class Context { + /** + * Adds the given element to the output {@code PCollection} with the given tag at the given + * timestamp in the given window. + * + *

Once passed to {@code output} the element should not be modified in any way. + * + *

Note: A splittable {@link DoFn} is not allowed to output from the {@link + * FinishBundle} method. + */ + public abstract void output( + TupleTag tag, T output, Instant timestamp, BoundedWindow window); + } + /** + * Information accessible to all methods in this {@link DoFn} where the context is in some window. + */ + public abstract class WindowedContext { /** - * Returns the {@code PipelineOptions} specified with the - * {@link org.apache.beam.sdk.runners.PipelineRunner} - * invoking this {@code DoFn}. The {@code PipelineOptions} will - * be the default running via {@link DoFnTester}. + * Returns the {@code PipelineOptions} specified with the {@link + * org.apache.beam.sdk.runners.PipelineRunner} invoking this {@code DoFn}. The {@code + * PipelineOptions} will be the default running via {@link DoFnTester}. */ public abstract PipelineOptions getPipelineOptions(); @@ -205,7 +252,7 @@ public abstract void outputWithTimestamp( /** * Information accessible when running a {@link DoFn.ProcessElement} method. */ - public abstract class ProcessContext extends Context { + public abstract class ProcessContext extends WindowedContext { /** * Returns the input element to be processed. @@ -215,7 +262,6 @@ public abstract class ProcessContext extends Context { */ public abstract InputT element(); - /** * Returns the value of the side input. * @@ -257,7 +303,7 @@ public abstract class ProcessContext extends Context { /** * Information accessible when running a {@link DoFn.OnTimer} method. */ - public abstract class OnTimerContext extends Context { + public abstract class OnTimerContext extends WindowedContext { /** * Returns the timestamp of the current timer. @@ -277,7 +323,7 @@ public abstract class OnTimerContext extends Context { /** * Returns the allowed timestamp skew duration, which is the maximum duration that timestamps can - * be shifted backward in {@link DoFn.Context#outputWithTimestamp}. + * be shifted backward in {@link WindowedContext#outputWithTimestamp}. * *

The default value is {@code Duration.ZERO}, in which case timestamps can only be shifted * forward to future. For infinite skew, return {@code Duration.millis(Long.MAX_VALUE)}. @@ -449,14 +495,9 @@ public interface OutputReceiver { * Annotation for the method to use to prepare an instance for processing a batch of elements. * The method annotated with this must satisfy the following constraints: *

    - *
  • It must have exactly one argument. - *
  • Its first (and only) argument must be a {@link DoFn.Context}. + *
  • It must have exactly zero or one arguments. + *
  • If it has any arguments, its only argument must be a {@link DoFn.StartBundleContext}. *
- * - *

A simple method declaration would look like: - * - * public void setup(DoFn.Context c) { .. } - * */ @Documented @Retention(RetentionPolicy.RUNTIME) @@ -524,8 +565,8 @@ public interface OutputReceiver { * Annotation for the method to use to finish processing a batch of elements. * The method annotated with this must satisfy the following constraints: *

    - *
  • It must have at least one argument. - *
  • Its first (and only) argument must be a {@link DoFn.Context}. + *
  • It must have exactly zero or one arguments. + *
  • If it has any arguments, its only argument must be a {@link DoFn.FinishBundleContext}. *
*/ @Documented diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index 70fb0ae9cb0b..ca7427c9bc64 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -218,9 +218,8 @@ public void startBundle() throws Exception { if (state == State.UNINITIALIZED) { initializeState(); } - TestContext context = new TestContext(); try { - fnInvoker.invokeStartBundle(context); + fnInvoker.invokeStartBundle(new TestStartBundleContext()); } catch (UserCodeException e) { unwrapUserCodeException(e); } @@ -291,9 +290,17 @@ public BoundedWindow window() { } @Override - public DoFn.Context context(DoFn doFn) { + public DoFn.StartBundleContext startBundleContext( + DoFn doFn) { throw new UnsupportedOperationException( - "Not expected to access DoFn.Context from @ProcessElement"); + "Not expected to access DoFn.StartBundleContext from @ProcessElement"); + } + + @Override + public DoFn.FinishBundleContext finishBundleContext( + DoFn doFn) { + throw new UnsupportedOperationException( + "Not expected to access DoFn.FinishBundleContext from @ProcessElement"); } @Override @@ -303,8 +310,7 @@ public DoFn.ProcessContext processContext(DoFn @Override public OnTimerContext onTimerContext(DoFn doFn) { - throw new UnsupportedOperationException( - "DoFnTester doesn't support timers yet."); + throw new UnsupportedOperationException("DoFnTester doesn't support timers yet."); } @Override @@ -344,7 +350,7 @@ public void finishBundle() throws Exception { "Must be inside bundle to call finishBundle, but was: %s", state); try { - fnInvoker.invokeFinishBundle(new TestContext()); + fnInvoker.invokeFinishBundle(new TestFinishBundleContext()); } catch (UserCodeException e) { unwrapUserCodeException(e); } @@ -522,8 +528,9 @@ public TupleTag getMainOutputTag() { return mainOutputTag; } - private class TestContext extends DoFn.Context { - TestContext() { + private class TestStartBundleContext extends DoFn.StartBundleContext { + + private TestStartBundleContext() { fn.super(); } @@ -531,30 +538,33 @@ private class TestContext extends DoFn.Context { public PipelineOptions getPipelineOptions() { return options; } + } - @Override - public void output(OutputT output) { - throwUnsupportedOutputFromBundleMethods(); + private class TestFinishBundleContext extends DoFn.FinishBundleContext { + + private TestFinishBundleContext() { + fn.super(); } - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - throwUnsupportedOutputFromBundleMethods(); + private void throwUnsupportedOutputFromBundleMethods() { + throw new UnsupportedOperationException( + "DoFnTester doesn't support output from bundle methods"); } @Override - public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - throwUnsupportedOutputFromBundleMethods(); + public PipelineOptions getPipelineOptions() { + return options; } @Override - public void output(TupleTag tag, T output) { + public void output( + OutputT output, Instant timestamp, BoundedWindow window) { throwUnsupportedOutputFromBundleMethods(); } - private void throwUnsupportedOutputFromBundleMethods() { - throw new UnsupportedOperationException( - "DoFnTester doesn't support output from bundle methods"); + @Override + public void output(TupleTag tag, T output, Instant timestamp, BoundedWindow window) { + throwUnsupportedOutputFromBundleMethods(); } } @@ -564,12 +574,10 @@ public DoFn.ProcessContext createProcessContext( } private class TestProcessContext extends DoFn.ProcessContext { - private final TestContext context; private final ValueInSingleWindow element; private TestProcessContext(ValueInSingleWindow element) { fn.super(); - this.context = new TestContext(); this.element = element; } @@ -611,7 +619,7 @@ public void updateWatermark(Instant watermark) { @Override public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); + return options; } @Override @@ -634,6 +642,12 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp getMutableOutput(tag) .add(ValueInSingleWindow.of(output, timestamp, element.getWindow(), element.getPane())); } + + private void throwUnsupportedOutputFromBundleMethods() { + throw new UnsupportedOperationException( + "DoFnTester doesn't support output from bundle methods"); + } + } @Override 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 564baf4e8b3b..e1321151ada0 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 @@ -195,7 +195,7 @@ public void onTimerCallback( } private void flushBatch( - Context c, + WindowedContext c, ValueState key, BagState batch, CombiningState numElementsInBatch) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index 73d78c965e92..7dd2cdd53ce8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -33,6 +33,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.DoFn.WindowedContext; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; import org.apache.beam.sdk.transforms.display.DisplayData.ItemSpec; @@ -191,8 +192,8 @@ * necessarily need to be explicitly specified, even if the {@link DoFn} * generates them. Within the {@link DoFn}, an element is added to the * main output {@link PCollection} as normal, using - * {@link DoFn.Context#output(Object)}, while an element is added to any additional output - * {@link PCollection} using {@link DoFn.Context#output(TupleTag, Object)}. For example: + * {@link WindowedContext#output(Object)}, while an element is added to any additional output + * {@link PCollection} using {@link WindowedContext#output(TupleTag, Object)}. For example: * *
{@code
  * PCollection words = ...;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
index 6bef4df68648..2b6d5be1c595 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
@@ -67,10 +67,11 @@
 import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.OnTimerMethod;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.Cases;
-import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.ContextParameter;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.FinishBundleContextParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.OnTimerContextParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.ProcessContextParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.RestrictionTrackerParameter;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.StartBundleContextParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.StateParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.TimerParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.WindowParameter;
@@ -84,6 +85,8 @@
 public class ByteBuddyDoFnInvokerFactory implements DoFnInvokerFactory {
 
   public static final String CONTEXT_PARAMETER_METHOD = "context";
+  public static final String START_BUNDLE_CONTEXT_PARAMETER_METHOD = "startBundleContext";
+  public static final String FINISH_BUNDLE_CONTEXT_PARAMETER_METHOD = "finishBundleContext";
   public static final String PROCESS_CONTEXT_PARAMETER_METHOD = "processContext";
   public static final String ON_TIMER_CONTEXT_PARAMETER_METHOD = "onTimerContext";
   public static final String WINDOW_PARAMETER_METHOD = "window";
@@ -557,11 +560,21 @@ static StackManipulation getExtraContextParameter(
         new Cases() {
 
           @Override
-          public StackManipulation dispatch(ContextParameter p) {
+          public StackManipulation dispatch(StartBundleContextParameter p) {
             return new StackManipulation.Compound(
                 pushDelegate,
                 MethodInvocation.invoke(
-                    getExtraContextFactoryMethodDescription(CONTEXT_PARAMETER_METHOD, DoFn.class)));
+                    getExtraContextFactoryMethodDescription(
+                        START_BUNDLE_CONTEXT_PARAMETER_METHOD, DoFn.class)));
+          }
+
+          @Override
+          public StackManipulation dispatch(FinishBundleContextParameter p) {
+            return new StackManipulation.Compound(
+                pushDelegate,
+                MethodInvocation.invoke(
+                    getExtraContextFactoryMethodDescription(
+                        FINISH_BUNDLE_CONTEXT_PARAMETER_METHOD, DoFn.class)));
           }
 
           @Override
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java
index 0fbcc84cfd9a..3c44afe7b98f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java
@@ -41,10 +41,10 @@ public interface DoFnInvoker {
   void invokeSetup();
 
   /** Invoke the {@link DoFn.StartBundle} method on the bound {@link DoFn}. */
-  void invokeStartBundle(DoFn.Context c);
+  void invokeStartBundle(DoFn.StartBundleContext c);
 
   /** Invoke the {@link DoFn.FinishBundle} method on the bound {@link DoFn}. */
-  void invokeFinishBundle(DoFn.Context c);
+  void invokeFinishBundle(DoFn.FinishBundleContext c);
 
   /** Invoke the {@link DoFn.Teardown} method on the bound {@link DoFn}. */
   void invokeTeardown();
@@ -100,8 +100,11 @@ interface ArgumentProvider {
      */
     BoundedWindow window();
 
-    /** Provide a {@link DoFn.Context} to use with the given {@link DoFn}. */
-    DoFn.Context context(DoFn doFn);
+    /** Provide a {@link DoFn.StartBundleContext} to use with the given {@link DoFn}. */
+    DoFn.StartBundleContext startBundleContext(DoFn doFn);
+
+    /** Provide a {@link DoFn.FinishBundleContext} to use with the given {@link DoFn}. */
+    DoFn.FinishBundleContext finishBundleContext(DoFn doFn);
 
     /** Provide a {@link DoFn.ProcessContext} to use with the given {@link DoFn}. */
     DoFn.ProcessContext processContext(DoFn doFn);
@@ -135,7 +138,13 @@ public BoundedWindow window() {
     }
 
     @Override
-    public DoFn.Context context(DoFn doFn) {
+    public DoFn.StartBundleContext startBundleContext(DoFn doFn) {
+      return null;
+    }
+
+    @Override
+    public DoFn.FinishBundleContext finishBundleContext(
+        DoFn doFn) {
       return null;
     }
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
index 1be741f2db39..3219f9680a55 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
@@ -177,8 +177,10 @@ private Parameter() {}
     public  ResultT match(Cases cases) {
       // This could be done with reflection, but since the number of cases is small and known,
       // they are simply inlined.
-      if (this instanceof ContextParameter) {
-        return cases.dispatch((ContextParameter) this);
+      if (this instanceof StartBundleContextParameter) {
+        return cases.dispatch((StartBundleContextParameter) this);
+      } else if (this instanceof FinishBundleContextParameter) {
+        return cases.dispatch((FinishBundleContextParameter) this);
       } else if (this instanceof ProcessContextParameter) {
         return cases.dispatch((ProcessContextParameter) this);
       } else if (this instanceof OnTimerContextParameter) {
@@ -202,7 +204,8 @@ public  ResultT match(Cases cases) {
      * An interface for destructuring a {@link Parameter}.
      */
     public interface Cases {
-      ResultT dispatch(ContextParameter p);
+      ResultT dispatch(StartBundleContextParameter p);
+      ResultT dispatch(FinishBundleContextParameter p);
       ResultT dispatch(ProcessContextParameter p);
       ResultT dispatch(OnTimerContextParameter p);
       ResultT dispatch(WindowParameter p);
@@ -218,7 +221,12 @@ abstract class WithDefault implements Cases {
         protected abstract ResultT dispatchDefault(Parameter p);
 
         @Override
-        public ResultT dispatch(ContextParameter p) {
+        public ResultT dispatch(StartBundleContextParameter p) {
+          return dispatchDefault(p);
+        }
+
+        @Override
+        public ResultT dispatch(FinishBundleContextParameter p) {
           return dispatchDefault(p);
         }
 
@@ -255,18 +263,15 @@ public ResultT dispatch(TimerParameter p) {
     }
 
     // These parameter descriptors are constant
-    private static final ContextParameter CONTEXT_PARAMETER =
-        new AutoValue_DoFnSignature_Parameter_ContextParameter();
+    private static final StartBundleContextParameter START_BUNDLE_CONTEXT_PARAMETER =
+        new AutoValue_DoFnSignature_Parameter_StartBundleContextParameter();
+    private static final FinishBundleContextParameter FINISH_BUNDLE_CONTEXT_PARAMETER =
+        new AutoValue_DoFnSignature_Parameter_FinishBundleContextParameter();
     private static final ProcessContextParameter PROCESS_CONTEXT_PARAMETER =
           new AutoValue_DoFnSignature_Parameter_ProcessContextParameter();
     private static final OnTimerContextParameter ON_TIMER_CONTEXT_PARAMETER =
         new AutoValue_DoFnSignature_Parameter_OnTimerContextParameter();
 
-    /** Returns a {@link ContextParameter}. */
-    public static ContextParameter context() {
-      return CONTEXT_PARAMETER;
-    }
-
     /** Returns a {@link ProcessContextParameter}. */
     public static ProcessContextParameter processContext() {
       return PROCESS_CONTEXT_PARAMETER;
@@ -301,13 +306,22 @@ public static TimerParameter timerParameter(TimerDeclaration decl) {
     }
 
     /**
-     * Descriptor for a {@link Parameter} of type {@link DoFn.Context}.
+     * Descriptor for a {@link Parameter} of type {@link DoFn.StartBundleContext}.
+     *
+     * 

All such descriptors are equal. + */ + @AutoValue + public abstract static class StartBundleContextParameter extends Parameter { + StartBundleContextParameter() {} + } + /** + * Descriptor for a {@link Parameter} of type {@link DoFn.FinishBundleContext}. * *

All such descriptors are equal. */ @AutoValue - public abstract static class ContextParameter extends Parameter { - ContextParameter() {} + public abstract static class FinishBundleContextParameter extends Parameter { + FinishBundleContextParameter() {} } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java index 666c7f69b3b7..bac3befc36df 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -346,14 +346,14 @@ private static DoFnSignature parseSignature(Class> fnClass) if (startBundleMethod != null) { ErrorReporter startBundleErrors = errors.forMethod(DoFn.StartBundle.class, startBundleMethod); signatureBuilder.setStartBundle( - analyzeBundleMethod(startBundleErrors, fnT, startBundleMethod, inputT, outputT)); + analyzeStartBundleMethod(startBundleErrors, fnT, startBundleMethod, inputT, outputT)); } if (finishBundleMethod != null) { ErrorReporter finishBundleErrors = errors.forMethod(DoFn.FinishBundle.class, finishBundleMethod); signatureBuilder.setFinishBundle( - analyzeBundleMethod(finishBundleErrors, fnT, finishBundleMethod, inputT, outputT)); + analyzeFinishBundleMethod(finishBundleErrors, fnT, finishBundleMethod, inputT, outputT)); } if (setupMethod != null) { @@ -607,12 +607,25 @@ TypeDescriptor.ProcessContext> doFnProcessContextTypeOf( } /** - * Generates a {@link TypeDescriptor} for {@code DoFn.Context} given {@code - * InputT} and {@code OutputT}. + * Generates a {@link TypeDescriptor} for {@code DoFn.StartBundleContext} given + * {@code InputT} and {@code OutputT}. + */ + private static + TypeDescriptor.StartBundleContext> doFnStartBundleContextTypeOf( + TypeDescriptor inputT, TypeDescriptor outputT) { + return new TypeDescriptor.StartBundleContext>() {}.where( + new TypeParameter() {}, inputT) + .where(new TypeParameter() {}, outputT); + } + + /** + * Generates a {@link TypeDescriptor} for {@code DoFn.FinishBundleContext} given + * {@code InputT} and {@code OutputT}. */ - private static TypeDescriptor.Context> doFnContextTypeOf( - TypeDescriptor inputT, TypeDescriptor outputT) { - return new TypeDescriptor.Context>() {}.where( + private static + TypeDescriptor.FinishBundleContext> doFnFinishBundleContextTypeOf( + TypeDescriptor inputT, TypeDescriptor outputT) { + return new TypeDescriptor.FinishBundleContext>() {}.where( new TypeParameter() {}, inputT) .where(new TypeParameter() {}, outputT); } @@ -752,7 +765,6 @@ private static Parameter analyzeExtraParameter( TypeDescriptor outputT) { TypeDescriptor expectedProcessContextT = doFnProcessContextTypeOf(inputT, outputT); - TypeDescriptor expectedContextT = doFnContextTypeOf(inputT, outputT); TypeDescriptor expectedOnTimerContextT = doFnOnTimerContextTypeOf(inputT, outputT); TypeDescriptor paramT = param.getType(); @@ -765,11 +777,6 @@ private static Parameter analyzeExtraParameter( "ProcessContext argument must have type %s", formatType(expectedProcessContextT)); return Parameter.processContext(); - } else if (rawType.equals(DoFn.Context.class)) { - paramErrors.checkArgument(paramT.equals(expectedContextT), - "Context argument must have type %s", - formatType(expectedContextT)); - return Parameter.context(); } else if (rawType.equals(DoFn.OnTimerContext.class)) { paramErrors.checkArgument( paramT.equals(expectedOnTimerContextT), @@ -921,17 +928,36 @@ private static TypeDescriptor getWindowType( } @VisibleForTesting - static DoFnSignature.BundleMethod analyzeBundleMethod( + static DoFnSignature.BundleMethod analyzeStartBundleMethod( + ErrorReporter errors, + TypeDescriptor> fnT, + Method m, + TypeDescriptor inputT, + TypeDescriptor outputT) { + errors.checkArgument(void.class.equals(m.getReturnType()), "Must return void"); + TypeDescriptor expectedContextT = doFnStartBundleContextTypeOf(inputT, outputT); + Type[] params = m.getGenericParameterTypes(); + errors.checkArgument( + params.length == 0 + || (params.length == 1 && fnT.resolveType(params[0]).equals(expectedContextT)), + "Must take a single argument of type %s", + formatType(expectedContextT)); + return DoFnSignature.BundleMethod.create(m); + } + + @VisibleForTesting + static DoFnSignature.BundleMethod analyzeFinishBundleMethod( ErrorReporter errors, TypeDescriptor> fnT, Method m, TypeDescriptor inputT, TypeDescriptor outputT) { errors.checkArgument(void.class.equals(m.getReturnType()), "Must return void"); - TypeDescriptor expectedContextT = doFnContextTypeOf(inputT, outputT); + TypeDescriptor expectedContextT = doFnFinishBundleContextTypeOf(inputT, outputT); Type[] params = m.getGenericParameterTypes(); errors.checkArgument( - params.length == 1 && fnT.resolveType(params[0]).equals(expectedContextT), + params.length == 0 + || (params.length == 1 && fnT.resolveType(params[0]).equals(expectedContextT)), "Must take a single argument of type %s", formatType(expectedContextT)); return DoFnSignature.BundleMethod.create(m); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java index b43c7a3a7836..faf3ca93612c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java @@ -30,6 +30,7 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.WindowedContext; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.util.VarInt; @@ -70,7 +71,7 @@ public final class PaneInfo { * definitions: *

    *
  1. We'll call a pipeline 'simple' if it does not use - * {@link DoFn.Context#outputWithTimestamp} in + * {@link WindowedContext#outputWithTimestamp} in * any {@link DoFn}, and it uses the same * {@link org.apache.beam.sdk.transforms.windowing.Window#withAllowedLateness} * argument value on all windows (or uses the default of {@link org.joda.time.Duration#ZERO}). diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java index 8077c27cd7b2..084c44508a75 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java @@ -210,7 +210,7 @@ private PipelineResult runPipelineWithMetrics() { Distribution bundleDist = Metrics.distribution(MetricsTest.class, "bundle"); @StartBundle - public void startBundle(Context c) { + public void startBundle() { bundleDist.update(10L); } @@ -226,7 +226,7 @@ public void processElement(ProcessContext c) { } @DoFn.FinishBundle - public void finishBundle(Context c) { + public void finishBundle() { bundleDist.update(40L); } })) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index d609d0e5f257..a980f87b5c74 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -195,12 +195,12 @@ public void process(ProcessContext context) { } @StartBundle - public void startBundle(Context context) { + public void startBundle() { ++numStartBundleCalls; } @FinishBundle - public void finishBundle(Context context) { + public void finishBundle() { ++numFinishBundleCalls; } } @@ -397,7 +397,7 @@ public void setup() { } @StartBundle - public void startBundle(Context c) { + public void startBundle() { checkState(state == LifecycleState.SET_UP, "Wrong state: %s", state); state = LifecycleState.INSIDE_BUNDLE; startBundleCalls.inc(); @@ -412,7 +412,7 @@ public void processElement(ProcessContext c) throws Exception { } @FinishBundle - public void finishBundle(Context c) { + public void finishBundle() { checkState(state == LifecycleState.INSIDE_BUNDLE, "Wrong state: %s", state); state = LifecycleState.SET_UP; finishBundleCalls.inc(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java index fda8947f067d..849b874852d5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java @@ -87,7 +87,7 @@ public void setup() { } @StartBundle - public void startBundle(Context c) { + public void startBundle() { assertThat("setup should have been called", setupCalled, is(true)); assertThat( "Even number of startBundle and finishBundle calls in startBundle", @@ -108,7 +108,7 @@ public void processElement(ProcessContext c) throws Exception { } @FinishBundle - public void finishBundle(Context c) { + public void finishBundle() { assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); assertThat( "there should be one bundle that has been started but not finished", @@ -167,7 +167,7 @@ public void before() { } @StartBundle - public void begin(Context c) { + public void begin() { assertThat("setup should have been called", setupCalled, is(true)); assertThat("Even number of startBundle and finishBundle calls in startBundle", startBundleCalls, @@ -186,7 +186,7 @@ public void process(ProcessContext c) throws Exception { } @FinishBundle - public void end(Context c) { + public void end() { assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); assertThat("there should be one bundle that has been started but not finished", startBundleCalls, @@ -354,7 +354,7 @@ public void setup() throws Exception { } @StartBundle - public void startBundle(Context c) throws Exception { + public void startBundle() throws Exception { throwIfNecessary(MethodForException.START_BUNDLE); } @@ -364,7 +364,7 @@ public void processElement(ProcessContext c) throws Exception { } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle() throws Exception { throwIfNecessary(MethodForException.FINISH_BUNDLE); } @@ -401,7 +401,7 @@ public void before() throws Exception { } @StartBundle - public void preBundle(Context c) throws Exception { + public void preBundle() throws Exception { throwIfNecessary(MethodForException.START_BUNDLE); } @@ -411,7 +411,7 @@ public void perElement(ProcessContext c) throws Exception { } @FinishBundle - public void postBundle(Context c) throws Exception { + public void postBundle() throws Exception { throwIfNecessary(MethodForException.FINISH_BUNDLE); } 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 1c919d4fb659..56051a69689d 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 @@ -170,12 +170,11 @@ public void prepare() { } @StartBundle - public void startBundle(Context c) { + public void startBundle() { assertThat(state, anyOf(equalTo(State.UNSTARTED), equalTo(State.FINISHED))); state = State.STARTED; - outputToAll(c, "started"); } @ProcessElement @@ -187,18 +186,17 @@ public void processElement(ProcessContext c) { } @FinishBundle - public void finishBundle(Context c) { + public void finishBundle(FinishBundleContext c) { assertThat(state, anyOf(equalTo(State.STARTED), equalTo(State.PROCESSING))); state = State.FINISHED; - outputToAll(c, "finished"); - } - - private void outputToAll(Context c, String value) { - c.output(value); + c.output("finished", BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE); for (TupleTag additionalOutputTupleTag : additionalOutputTupleTags) { - c.output(additionalOutputTupleTag, - additionalOutputTupleTag.getId() + ": " + value); + c.output( + additionalOutputTupleTag, + additionalOutputTupleTag.getId() + ": " + "finished", + BoundedWindow.TIMESTAMP_MIN_VALUE, + GlobalWindow.INSTANCE); } } @@ -220,7 +218,7 @@ private void outputToAllWithSideInputs(ProcessContext c, String value) { static class TestStartBatchErrorDoFn extends DoFn { @StartBundle - public void startBundle(Context c) { + public void startBundle() { throw new RuntimeException("test error in initialize"); } @@ -244,7 +242,7 @@ public void processElement(ProcessContext c) { } @FinishBundle - public void finishBundle(Context c) { + public void finishBundle(FinishBundleContext c) { throw new RuntimeException("test error in finalize"); } } @@ -1156,13 +1154,10 @@ public HasExpectedOutput inOrder() { @Override public Void apply(Iterable outputs) { - List starteds = new ArrayList<>(); List processeds = new ArrayList<>(); List finisheds = new ArrayList<>(); for (String output : outputs) { - if (output.contains("started")) { - starteds.add(output); - } else if (output.contains("finished")) { + if (output.contains("finished")) { finisheds.add(output); } else { processeds.add(output); @@ -1196,10 +1191,6 @@ public Void apply(Iterable outputs) { assertThat(processeds, contains(expectedProcessedsArray)); } - assertEquals(starteds.size(), finisheds.size()); - for (String started : starteds) { - assertEquals(additionalOutputPrefix + "started", started); - } for (String finished : finisheds) { assertEquals(additionalOutputPrefix + "finished", finished); } @@ -1483,7 +1474,6 @@ public Void apply(Iterable input) { private static class Checker implements SerializableFunction, Void> { @Override public Void apply(Iterable input) { - boolean foundStart = false; boolean foundElement = false; boolean foundFinish = false; for (String str : input) { @@ -1492,17 +1482,12 @@ public Void apply(Iterable input) { throw new AssertionError("Received duplicate element"); } foundElement = true; - } else if (str.equals("start:2:2")) { - foundStart = true; } else if (str.equals("finish:3:3")) { foundFinish = true; } else { throw new AssertionError("Got unexpected value: " + str); } } - if (!foundStart) { - throw new AssertionError("Missing \"start:2:2\""); - } if (!foundElement) { throw new AssertionError("Missing \"elem:1:1\""); } @@ -1518,19 +1503,14 @@ public Void apply(Iterable input) { @Category(ValidatesRunner.class) public void testWindowingInStartAndFinishBundle() { + final FixedWindows windowFn = FixedWindows.of(Duration.millis(1)); PCollection output = pipeline .apply(Create.timestamped(TimestampedValue.of("elem", new Instant(1)))) - .apply(Window.into(FixedWindows.of(Duration.millis(1)))) + .apply(Window.into(windowFn)) .apply( ParDo.of( new DoFn() { - @StartBundle - public void startBundle(Context c) { - c.outputWithTimestamp("start", new Instant(2)); - System.out.println("Start: 2"); - } - @ProcessElement public void processElement(ProcessContext c) { c.output(c.element()); @@ -1539,8 +1519,9 @@ public void processElement(ProcessContext c) { } @FinishBundle - public void finishBundle(Context c) { - c.outputWithTimestamp("finish", new Instant(3)); + public void finishBundle(FinishBundleContext c) { + Instant ts = new Instant(3); + c.output("finish", ts, windowFn.assignWindow(ts)); System.out.println("Finish: 3"); } })) @@ -1551,30 +1532,6 @@ public void finishBundle(Context c) { pipeline.run(); } - @Test - @Category(NeedsRunner.class) - public void testWindowingInStartBundleException() { - - pipeline - .apply(Create.timestamped(TimestampedValue.of("elem", new Instant(1)))) - .apply(Window.into(FixedWindows.of(Duration.millis(1)))) - .apply( - ParDo.of( - new DoFn() { - @StartBundle - public void startBundle(Context c) { - c.output("start"); - } - - @ProcessElement - public void processElement(ProcessContext c) { - c.output(c.element()); - } - })); - - thrown.expectMessage("WindowFn attempted to access input timestamp when none was available"); - pipeline.run(); - } @Test public void testDoFnDisplayData() { DoFn fn = new DoFn() { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java index a0f1fd3573f4..02a44d2b907e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java @@ -405,13 +405,13 @@ public void setUp() { } @StartBundle - public void startBundle(Context c) { + public void startBundle() { assertEquals(State.OUTSIDE_BUNDLE, state); state = State.INSIDE_BUNDLE; } @FinishBundle - public void finishBundle(Context c) { + public void finishBundle() { assertEquals(State.INSIDE_BUNDLE, state); state = State.OUTSIDE_BUNDLE; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index c16eea2ba5c9..fe96e87951a4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -71,6 +71,8 @@ public class DoFnInvokersTest { @Rule public ExpectedException thrown = ExpectedException.none(); + @Mock private DoFn.StartBundleContext mockStartBundleContext; + @Mock private DoFn.FinishBundleContext mockFinishBundleContext; @Mock private DoFn.ProcessContext mockProcessContext; @Mock private IntervalWindow mockWindow; @Mock private DoFnInvoker.ArgumentProvider mockArgumentProvider; @@ -79,6 +81,10 @@ public class DoFnInvokersTest { public void setUp() { MockitoAnnotations.initMocks(this); when(mockArgumentProvider.window()).thenReturn(mockWindow); + when(mockArgumentProvider.startBundleContext(Matchers.any())) + .thenReturn(mockStartBundleContext); + when(mockArgumentProvider.finishBundleContext(Matchers.any())) + .thenReturn(mockFinishBundleContext); when(mockArgumentProvider.processContext(Matchers.any())).thenReturn(mockProcessContext); } @@ -233,10 +239,10 @@ class MockFn extends DoFn { public void processElement(ProcessContext c) {} @StartBundle - public void startBundle(Context c) {} + public void startBundle(StartBundleContext c) {} @FinishBundle - public void finishBundle(Context c) {} + public void finishBundle(FinishBundleContext c) {} @Setup public void before() {} @@ -247,12 +253,12 @@ public void after() {} MockFn fn = mock(MockFn.class); DoFnInvoker invoker = DoFnInvokers.invokerFor(fn); invoker.invokeSetup(); - invoker.invokeStartBundle(mockProcessContext); - invoker.invokeFinishBundle(mockProcessContext); + invoker.invokeStartBundle(mockStartBundleContext); + invoker.invokeFinishBundle(mockFinishBundleContext); invoker.invokeTeardown(); verify(fn).before(); - verify(fn).startBundle(mockProcessContext); - verify(fn).finishBundle(mockProcessContext); + verify(fn).startBundle(mockStartBundleContext); + verify(fn).finishBundle(mockFinishBundleContext); verify(fn).after(); } @@ -601,7 +607,7 @@ public void testStartBundleException() throws Exception { DoFnInvokers.invokerFor( new DoFn() { @StartBundle - public void startBundle(@SuppressWarnings("unused") Context c) { + public void startBundle(@SuppressWarnings("unused") StartBundleContext c) { throw new IllegalArgumentException("bogus"); } @@ -619,7 +625,7 @@ public void testFinishBundleException() throws Exception { DoFnInvokers.invokerFor( new DoFn() { @FinishBundle - public void finishBundle(@SuppressWarnings("unused") Context c) { + public void finishBundle(@SuppressWarnings("unused") FinishBundleContext c) { throw new IllegalArgumentException("bogus"); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java index d6cc4f6a51f1..f099d5d1b846 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java @@ -76,13 +76,14 @@ public void process(ProcessContext c) {} @Test public void testBadExtraContext() throws Exception { thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Must take a single argument of type DoFn.Context"); + thrown.expectMessage( + "Must take a single argument of type DoFn.StartBundleContext"); - DoFnSignatures.analyzeBundleMethod( + DoFnSignatures.analyzeStartBundleMethod( errors(), TypeDescriptor.of(FakeDoFn.class), new DoFnSignaturesTestUtils.AnonymousMethod() { - void method(DoFn.Context c, int n) {} + void method(DoFn.StartBundleContext c, int n) {} }.getMethod(), TypeDescriptor.of(Integer.class), TypeDescriptor.of(String.class)); @@ -112,8 +113,8 @@ public void baz() {} public void testMultipleFinishBundleMethods() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Found multiple methods annotated with @FinishBundle"); - thrown.expectMessage("bar(Context)"); - thrown.expectMessage("baz(Context)"); + thrown.expectMessage("bar(FinishBundleContext)"); + thrown.expectMessage("baz(FinishBundleContext)"); thrown.expectMessage(getClass().getName() + "$"); DoFnSignatures.getSignature( new DoFn() { @@ -121,10 +122,10 @@ public void testMultipleFinishBundleMethods() throws Exception { public void foo(ProcessContext context) {} @FinishBundle - public void bar(Context context) {} + public void bar(FinishBundleContext context) {} @FinishBundle - public void baz(Context context) {} + public void baz(FinishBundleContext context) {} }.getClass()); } 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 1cdd087cfd47..6d5e230b9c20 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 @@ -71,6 +71,7 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; @@ -304,20 +305,21 @@ private static class TestDoFn extends DoFn { private static final TupleTag mainOutput = new TupleTag<>("mainOutput"); private static final TupleTag additionalOutput = new TupleTag<>("output"); - @StartBundle - public void startBundle(Context context) { - context.output("StartBundle"); - } + private BoundedWindow window; @ProcessElement - public void processElement(ProcessContext context) { + public void processElement(ProcessContext context, BoundedWindow window) { context.output("MainOutput" + context.element()); context.output(additionalOutput, "AdditionalOutput" + context.element()); + this.window = window; } @FinishBundle - public void finishBundle(Context context) { - context.output("FinishBundle"); + public void finishBundle(FinishBundleContext context) { + if (window != null) { + context.output("FinishBundle", window.maxTimestamp(), window); + window = null; + } } } @@ -411,7 +413,6 @@ public void testCreatingAndProcessingDoFn() throws Exception { finishFunctions::add); Iterables.getOnlyElement(startFunctions).run(); - assertThat(mainOutputValues, contains(valueInGlobalWindow("StartBundle"))); mainOutputValues.clear(); assertEquals(newConsumers.keySet(), diff --git a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java index 0a3b900c548b..f6ceef2286f5 100644 --- a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java +++ b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java @@ -25,7 +25,6 @@ import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; - import java.io.IOException; import java.io.Serializable; import java.net.MalformedURLException; @@ -40,7 +39,6 @@ import java.util.Map; import java.util.NoSuchElementException; import javax.annotation.Nullable; - import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -757,7 +755,7 @@ public void createClient() throws Exception { } @StartBundle - public void startBundle(Context context) throws Exception { + public void startBundle(StartBundleContext context) throws Exception { batch = new ArrayList<>(); currentBatchSizeBytes = 0; } @@ -769,12 +767,16 @@ public void processElement(ProcessContext context) throws Exception { currentBatchSizeBytes += document.getBytes().length; if (batch.size() >= spec.getMaxBatchSize() || currentBatchSizeBytes >= spec.getMaxBatchSizeBytes()) { - finishBundle(context); + flushBatch(); } } @FinishBundle - public void finishBundle(Context context) throws Exception { + public void finishBundle(FinishBundleContext context) throws Exception { + flushBatch(); + } + + private void flushBatch() throws IOException { if (batch.isEmpty()) { return; } 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 fd5f39655cb5..f267976f68dd 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 @@ -56,7 +56,7 @@ class StreamingWriteFn /** Prepares a target BigQuery table. */ @StartBundle - public void startBundle(Context context) { + public void startBundle() { tableRows = new HashMap<>(); uniqueIdsForTableRows = new HashMap<>(); } @@ -75,7 +75,7 @@ public void processElement(ProcessContext context) { /** Writes the accumulated rows into BigQuery with streaming API. */ @FinishBundle - public void finishBundle(Context context) throws Exception { + public void finishBundle(FinishBundleContext context) throws Exception { BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class); for (Map.Entry> entry : tableRows.entrySet()) { TableReference tableReference = BigQueryHelpers.parseTableSpec(entry.getKey()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java index 284691e49f55..cd88222da69f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java @@ -40,7 +40,7 @@ class TagWithUniqueIds private transient long sequenceNo = 0L; @StartBundle - public void startBundle(Context context) { + public void startBundle() { randomUUID = UUID.randomUUID().toString(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java index e90b974d973f..70aa135aab27 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,8 +49,10 @@ class WriteBundlesToFiles // Map from tablespec to a writer for that table. private transient Map writers; + private transient Map writerWindows; private final String stepUuid; + /** * The result of the {@link WriteBundlesToFiles} transform. Corresponds to a single output file, * and encapsulates the table it is destined to as well as the file byte size. @@ -110,14 +113,15 @@ public void verifyDeterministic() {} } @StartBundle - public void startBundle(Context c) { + public void startBundle() { // This must be done each bundle, as by default the {@link DoFn} might be reused between // bundles. this.writers = Maps.newHashMap(); + this.writerWindows = Maps.newHashMap(); } @ProcessElement - public void processElement(ProcessContext c) throws Exception { + public void processElement(ProcessContext c, BoundedWindow window) throws Exception { String tempFilePrefix = resolveTempLocation( c.getPipelineOptions().getTempLocation(), "BigQueryWriteTemp", stepUuid); TableRowWriter writer = writers.get(c.element().getKey()); @@ -125,6 +129,7 @@ public void processElement(ProcessContext c) throws Exception { writer = new TableRowWriter(tempFilePrefix); writer.open(UUID.randomUUID().toString()); writers.put(c.element().getKey(), writer); + writerWindows.put(c.element().getKey(), window); LOG.debug("Done opening writer {}", writer); } try { @@ -143,11 +148,15 @@ public void processElement(ProcessContext c) throws Exception { } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle(FinishBundleContext c) throws Exception { for (Map.Entry entry : writers.entrySet()) { TableRowWriter.Result result = entry.getValue().close(); - c.output(new Result<>(result.resourceId.toString(), result.byteSize, entry.getKey())); + c.output( + new Result<>(result.resourceId.toString(), result.byteSize, entry.getKey()), + writerWindows.get(entry.getKey()).maxTimestamp(), + writerWindows.get(entry.getKey())); } writers.clear(); + writerWindows.clear(); } } 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 69fac6836329..0e97c1254d93 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 @@ -572,7 +572,7 @@ public BigtableWriterFn(String tableId, } @StartBundle - public void startBundle(Context c) throws IOException { + public void startBundle(StartBundleContext c) throws IOException { if (bigtableWriter == null) { bigtableWriter = bigtableServiceFactory.apply( c.getPipelineOptions()).openForWriting(tableId); @@ -589,7 +589,7 @@ public void processElement(ProcessContext c) throws Exception { } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle() throws Exception { bigtableWriter.flush(); checkForFailures(); LOG.info("Wrote {} records", recordsWritten); 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 f6194296049a..fd4fccf0ba1d 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 @@ -704,7 +704,7 @@ static class GqlQueryTranslateFn extends DoFn, Query> { } @StartBundle - public void startBundle(Context c) throws Exception { + public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), v1Options.getProjectId()); } @@ -748,7 +748,7 @@ public SplitQueryFn(V1Options options, int numSplits) { } @StartBundle - public void startBundle(Context c) throws Exception { + public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), options.getProjectId(), options.getLocalhost()); querySplitter = datastoreFactory.getQuerySplitter(); @@ -821,7 +821,7 @@ public ReadFn(V1Options options) { } @StartBundle - public void startBundle(Context c) throws Exception { + public void startBundle(StartBundleContext c) throws Exception { datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), options.getProjectId(), options.getLocalhost()); } @@ -1145,7 +1145,7 @@ static class DatastoreWriterFn extends DoFn { } @StartBundle - public void startBundle(Context c) { + public void startBundle(StartBundleContext c) { datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId.get(), localhost); } @@ -1158,7 +1158,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(); } 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 e023ad0dca07..fa2d20f317ba 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 @@ -888,7 +888,7 @@ public class PubsubBoundedWriter extends DoFn { private transient PubsubClient pubsubClient; @StartBundle - public void startBundle(Context c) throws IOException { + public void startBundle(StartBundleContext c) throws IOException { this.output = new ArrayList<>(); // NOTE: idAttribute is ignored. this.pubsubClient = @@ -911,7 +911,7 @@ public void processElement(ProcessContext c) throws IOException { } @FinishBundle - public void finishBundle(Context c) throws IOException { + public void finishBundle() throws IOException { if (!output.isEmpty()) { publish(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java index 9d97e91f50bc..031d9a0ab14e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java @@ -255,7 +255,7 @@ private void publishBatch(List messages, int bytes) } @StartBundle - public void startBundle(Context c) throws Exception { + public void startBundle(StartBundleContext c) throws Exception { checkState(pubsubClient == null, "startBundle invoked without prior finishBundle"); pubsubClient = pubsubFactory.newClient(timestampAttribute, idAttribute, c.getPipelineOptions().as(PubsubOptions.class)); @@ -287,7 +287,7 @@ public void processElement(ProcessContext c) throws Exception { } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle() throws Exception { pubsubClient.close(); pubsubClient = null; } diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Write.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Write.java index 86a9246a3516..ef6556eb481d 100644 --- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Write.java +++ b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Write.java @@ -223,6 +223,7 @@ private class WriteBundles extends DoFn { // Writer that will write the records in this bundle. Lazily // initialized in processElement. private Writer writer = null; + private BoundedWindow window; private final PCollectionView> writeOperationView; WriteBundles(PCollectionView> writeOperationView) { @@ -243,6 +244,7 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except } else { writer.openUnwindowed(UUID.randomUUID().toString(), UNKNOWN_SHARDNUM, UNKNOWN_NUMSHARDS); } + this.window = window; LOG.debug("Done opening writer {} for operation {}", writer, writeOperationView); } try { @@ -265,12 +267,13 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle(FinishBundleContext c) throws Exception { if (writer != null) { WriteT result = writer.close(); - c.output(result); + c.output(result, window.maxTimestamp(), window); // Reset state in case of reuse. writer = null; + window = null; } } 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 eee892701123..3c42da91f27a 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 @@ -634,11 +634,6 @@ public void setup() throws Exception { recordsWritten = 0; } - @StartBundle - public void startBundle(Context c) throws Exception { - - } - @ProcessElement public void processElement(ProcessContext ctx) throws Exception { KV> record = ctx.element(); @@ -651,7 +646,7 @@ public void processElement(ProcessContext ctx) throws Exception { } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle() throws Exception { mutator.flush(); } diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java index 2d4823679fac..2eb53dd6361a 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java @@ -25,6 +25,7 @@ import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.ResultSet; +import java.sql.SQLException; import java.util.Random; import javax.annotation.Nullable; @@ -480,7 +481,7 @@ public void setup() throws Exception { } @StartBundle - public void startBundle(Context context) { + public void startBundle() { batchCount = 0; } @@ -495,12 +496,16 @@ public void processElement(ProcessContext context) throws Exception { batchCount++; if (batchCount >= DEFAULT_BATCH_SIZE) { - finishBundle(context); + executeBatch(); } } @FinishBundle - public void finishBundle(Context context) throws Exception { + public void finishBundle() throws Exception { + executeBatch(); + } + + private void executeBatch() throws SQLException { if (batchCount > 0) { preparedStatement.executeBatch(); connection.commit(); 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 813e0512d44f..4493e56d3fee 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 @@ -674,7 +674,7 @@ public WriterFn(Write spec) { } @StartBundle - public void startBundle(Context c) throws Exception { + public void startBundle() throws Exception { if (producer == null) { if (spec.getUsername() != null) { this.connection = @@ -703,13 +703,13 @@ public void processElement(ProcessContext ctx) throws Exception { TextMessage message = session.createTextMessage(value); producer.send(message); } catch (Exception t) { - finishBundle(null); + finishBundle(); throw t; } } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle() throws Exception { producer.close(); producer = null; session.close(); 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 8ab33d1c5b69..f4de76a56b47 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 @@ -1632,7 +1632,7 @@ public void processElement(ProcessContext ctx) throws Exception { } @FinishBundle - public void finishBundle(Context c) throws IOException { + public void finishBundle() throws IOException { producer.flush(); checkForFailures(); } diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java index 940d87534ac9..0868ed443c84 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java @@ -641,7 +641,7 @@ public void setup() throws Exception { } @StartBundle - public void startBundle(Context context) { + public void startBundle() { gridFsFile = gridfs.createFile(spec.filename()); if (spec.chunkSize() != null) { gridFsFile.setChunkSize(spec.chunkSize()); @@ -656,7 +656,7 @@ public void processElement(ProcessContext context) throws Exception { } @FinishBundle - public void finishBundle(Context context) throws Exception { + public void finishBundle() throws Exception { if (gridFsFile != null) { outputStream.flush(); outputStream.close(); diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java index f8edbf11a8e9..7236a50655e5 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java @@ -466,7 +466,7 @@ public void createMongoClient() throws Exception { } @StartBundle - public void startBundle(Context ctx) throws Exception { + public void startBundle() throws Exception { batch = new ArrayList<>(); } @@ -476,12 +476,16 @@ public void processElement(ProcessContext ctx) throws Exception { // before inserting (will assign an id). batch.add(new Document(ctx.element())); if (batch.size() >= spec.batchSize()) { - finishBundle(ctx); + flush(); } } @FinishBundle - public void finishBundle(Context ctx) throws Exception { + public void finishBundle() throws Exception { + flush(); + } + + private void flush() { MongoDatabase mongoDatabase = client.getDatabase(spec.database()); MongoCollection mongoCollection = mongoDatabase.getCollection(spec.collection()); From d9943a3cbd402872053f5482cf08cb3b70416bd4 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Thu, 4 May 2017 14:12:24 -0700 Subject: [PATCH 047/387] [BEAM-2170] PubsubMessageWithAttributesCoder should not NPE on messages without attributes --- .../PubsubMessageWithAttributesCoder.java | 14 ++++---- .../gcp/pubsub/PubsubUnboundedSinkTest.java | 34 ++++++++++++++++--- 2 files changed, 35 insertions(+), 13 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java index f70955da92e0..e061edc50106 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java @@ -31,10 +31,11 @@ /** A coder for PubsubMessage including attributes. */ public class PubsubMessageWithAttributesCoder extends CustomCoder { - private static final Coder PAYLOAD_CODER = - NullableCoder.of(ByteArrayCoder.of()); - private static final Coder> ATTRIBUTES_CODER = MapCoder.of( - StringUtf8Coder.of(), StringUtf8Coder.of()); + // A message's payload can not be null + private static final Coder PAYLOAD_CODER = ByteArrayCoder.of(); + // A message's attributes can be null. + private static final Coder> ATTRIBUTES_CODER = + NullableCoder.of(MapCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); public static Coder of(TypeDescriptor ignored) { return of(); @@ -46,10 +47,7 @@ public static PubsubMessageWithAttributesCoder of() { public void encode(PubsubMessage value, OutputStream outStream, Context context) throws IOException { - PAYLOAD_CODER.encode( - value.getPayload(), - outStream, - context.nested()); + PAYLOAD_CODER.encode(value.getPayload(), outStream, context.nested()); ATTRIBUTES_CODER.encode(value.getAttributeMap(), outStream, context); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java index cc3c85e17ff2..e32e9a86f400 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java @@ -33,7 +33,6 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubUnboundedSink.RecordIdMethod; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.CoderProperties; -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.DoFn; @@ -42,7 +41,6 @@ import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -96,7 +94,6 @@ public void saneCoder() throws Exception { } @Test - @Category(NeedsRunner.class) public void sendOneMessage() throws IOException { List outgoing = ImmutableList.of(new OutgoingMessage( @@ -123,7 +120,35 @@ public void sendOneMessage() throws IOException { } @Test - @Category(NeedsRunner.class) + public void sendOneMessageWithoutAttributes() throws IOException { + List outgoing = + ImmutableList.of( + new OutgoingMessage( + DATA.getBytes(), null /* attributes */, TIMESTAMP, getRecordId(DATA))); + try (PubsubTestClientFactory factory = + PubsubTestClient.createFactoryForPublish( + TOPIC, outgoing, ImmutableList.of())) { + PubsubUnboundedSink sink = + new PubsubUnboundedSink( + factory, + StaticValueProvider.of(TOPIC), + TIMESTAMP_ATTRIBUTE, + ID_ATTRIBUTE, + NUM_SHARDS, + 1 /* batchSize */, + 1 /* batchBytes */, + Duration.standardSeconds(2), + RecordIdMethod.DETERMINISTIC); + p.apply(Create.of(ImmutableList.of(DATA))) + .apply(ParDo.of(new Stamp(null /* attributes */))) + .apply(sink); + p.run(); + } + // The PubsubTestClientFactory will assert fail on close if the actual published + // message does not match the expected publish message. + } + + @Test public void sendMoreThanOneBatchByNumMessages() throws IOException { List outgoing = new ArrayList<>(); List data = new ArrayList<>(); @@ -152,7 +177,6 @@ public void sendMoreThanOneBatchByNumMessages() throws IOException { } @Test - @Category(NeedsRunner.class) public void sendMoreThanOneBatchByByteSize() throws IOException { List outgoing = new ArrayList<>(); List data = new ArrayList<>(); From c9b7fe443368badf6fd9fbd08f5234f17766c2cf Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Thu, 4 May 2017 14:40:08 -0700 Subject: [PATCH 048/387] Makes PubsubMessagePayloadOnlyCoder not require whole-stream context Now that PubsubIO.Read can directly read PubsubMessage's, they should be treated as first-class PCollection elements, and they can be encoded/decoded in any contexts. --- .../gcp/pubsub/PubsubMessagePayloadOnlyCoder.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessagePayloadOnlyCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessagePayloadOnlyCoder.java index 81c1a45b4d5b..d120f7225fe5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessagePayloadOnlyCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessagePayloadOnlyCoder.java @@ -17,17 +17,18 @@ */ package org.apache.beam.sdk.io.gcp.pubsub; -import static com.google.common.base.Preconditions.checkState; - import com.google.common.collect.ImmutableMap; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.util.StreamUtils; /** A coder for PubsubMessage treating the raw bytes being decoded as the message's payload. */ public class PubsubMessagePayloadOnlyCoder extends CustomCoder { + private static final Coder PAYLOAD_CODER = ByteArrayCoder.of(); + public static PubsubMessagePayloadOnlyCoder of() { return new PubsubMessagePayloadOnlyCoder(); } @@ -35,14 +36,12 @@ public static PubsubMessagePayloadOnlyCoder of() { @Override public void encode(PubsubMessage value, OutputStream outStream, Context context) throws IOException { - checkState(context.isWholeStream, "Expected to only be used in a whole-stream context"); - outStream.write(value.getPayload()); + PAYLOAD_CODER.encode(value.getPayload(), outStream, context); } @Override public PubsubMessage decode(InputStream inStream, Context context) throws IOException { - checkState(context.isWholeStream, "Expected to only be used in a whole-stream context"); return new PubsubMessage( - StreamUtils.getBytes(inStream), ImmutableMap.of()); + PAYLOAD_CODER.decode(inStream, context), ImmutableMap.of()); } } From 9d152498414ead3db77d64f49a2e90f4d4d255e0 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Thu, 4 May 2017 14:58:40 -0700 Subject: [PATCH 049/387] Moves coder choice into PubsubSource --- .../java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java | 9 +-------- .../beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java | 8 ++++---- 2 files changed, 5 insertions(+), 12 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 fa2d20f317ba..51da111ccd24 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 @@ -721,14 +721,7 @@ public PCollection expand(PBegin input) { getTimestampAttribute(), getIdAttribute(), getNeedsAttributes()); - return input - .getPipeline() - .apply(source) - .setCoder( - getNeedsAttributes() - ? PubsubMessageWithAttributesCoder.of() - : PubsubMessagePayloadOnlyCoder.of()) - .apply(MapElements.via(getParseFn())); + return input.apply(source).apply(MapElements.via(getParseFn())); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java index e5be71ba9e29..c2cbe73f484b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java @@ -1155,14 +1155,14 @@ public PubsubReader createReader( @Nullable @Override public Coder getCheckpointMarkCoder() { - @SuppressWarnings("unchecked") PubsubCheckpointCoder typedCoder = - (PubsubCheckpointCoder) CHECKPOINT_CODER; - return typedCoder; + return CHECKPOINT_CODER; } @Override public Coder getDefaultOutputCoder() { - return new PubsubMessageWithAttributesCoder(); + return outer.getNeedsAttributes() + ? PubsubMessageWithAttributesCoder.of() + : PubsubMessagePayloadOnlyCoder.of(); } @Override From 78e0accad907420675aefd0ac617bcf5d2a82619 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 2 May 2017 10:29:33 -0700 Subject: [PATCH 050/387] Move Java sdk.util.state to sdk.state --- .../translation/utils/ApexStateInternals.java | 20 ++++++++--------- .../utils/StateInternalsProxy.java | 4 ++-- .../utils/ApexStateInternalsTest.java | 12 +++++----- .../construction/PTransformMatchersTest.java | 6 ++--- .../beam/runners/core/DoFnAdapters.java | 2 +- .../runners/core/InMemoryStateInternals.java | 22 +++++++++---------- .../runners/core/MergingActiveWindowSet.java | 2 +- .../runners/core/MergingStateAccessor.java | 2 +- .../beam/runners/core/NonEmptyPanes.java | 4 ++-- ...oundedSplittableProcessElementInvoker.java | 2 +- .../beam/runners/core/PaneInfoTracker.java | 4 ++-- .../apache/beam/runners/core/ReduceFn.java | 2 +- .../runners/core/ReduceFnContextFactory.java | 6 ++--- .../beam/runners/core/SideInputHandler.java | 4 ++-- .../beam/runners/core/SimpleDoFnRunner.java | 4 ++-- .../beam/runners/core/SplittableParDo.java | 4 ++-- .../beam/runners/core/StateAccessor.java | 2 +- .../beam/runners/core/StateInternals.java | 4 ++-- .../beam/runners/core/StateMerging.java | 14 ++++++------ .../apache/beam/runners/core/StateTable.java | 4 ++-- .../apache/beam/runners/core/StateTag.java | 16 +++++++------- .../apache/beam/runners/core/StateTags.java | 20 ++++++++--------- .../beam/runners/core/StatefulDoFnRunner.java | 4 ++-- .../beam/runners/core/SystemReduceFn.java | 8 +++---- .../core/TestInMemoryStateInternals.java | 4 ++-- .../beam/runners/core/WatermarkHold.java | 4 ++-- ...fterDelayFromFirstElementStateMachine.java | 4 ++-- .../core/triggers/AfterPaneStateMachine.java | 2 +- .../TriggerStateMachineContextFactory.java | 2 +- .../triggers/TriggerStateMachineRunner.java | 2 +- .../core/InMemoryStateInternalsTest.java | 16 +++++++------- .../runners/core/StatefulDoFnRunnerTest.java | 6 ++--- .../CopyOnAccessInMemoryStateInternals.java | 18 +++++++-------- .../direct/StatefulParDoEvaluatorFactory.java | 2 +- ...opyOnAccessInMemoryStateInternalsTest.java | 14 ++++++------ .../runners/direct/EvaluationContextTest.java | 2 +- .../StatefulParDoEvaluatorFactoryTest.java | 6 ++--- .../flink/FlinkDetachedRunnerResult.java | 1 - .../types/EncodedValueSerializer.java | 2 -- .../wrappers/streaming/DoFnOperator.java | 2 +- .../state/FlinkBroadcastStateInternals.java | 20 ++++++++--------- .../state/FlinkKeyGroupStateInternals.java | 20 ++++++++--------- .../state/FlinkSplitStateInternals.java | 20 ++++++++--------- .../streaming/state/FlinkStateInternals.java | 20 ++++++++--------- .../flink/streaming/DoFnOperatorTest.java | 6 ++--- .../FlinkBroadcastStateInternalsTest.java | 10 ++++----- .../FlinkKeyGroupStateInternalsTest.java | 4 ++-- .../FlinkSplitStateInternalsTest.java | 4 ++-- .../streaming/FlinkStateInternalsTest.java | 12 +++++----- .../BatchStatefulParDoOverridesTest.java | 6 ++--- .../DataflowPipelineTranslatorTest.java | 6 ++--- .../spark/stateful/SparkStateInternals.java | 20 ++++++++--------- .../main/resources/beam/findbugs-filter.xml | 2 +- .../beam/sdk/{util => }/state/BagState.java | 2 +- .../sdk/{util => }/state/CombiningState.java | 2 +- .../sdk/{util => }/state/GroupingState.java | 2 +- .../beam/sdk/{util => }/state/MapState.java | 2 +- .../sdk/{util => }/state/ReadableState.java | 2 +- .../sdk/{util => }/state/ReadableStates.java | 2 +- .../beam/sdk/{util => }/state/SetState.java | 2 +- .../beam/sdk/{util => }/state/State.java | 2 +- .../sdk/{util => }/state/StateBinder.java | 2 +- .../sdk/{util => }/state/StateContext.java | 2 +- .../sdk/{util => }/state/StateContexts.java | 2 +- .../beam/sdk/{util => }/state/StateSpec.java | 2 +- .../beam/sdk/{util => }/state/StateSpecs.java | 2 +- .../beam/sdk/{util => }/state/ValueState.java | 2 +- .../{util => }/state/WatermarkHoldState.java | 2 +- .../sdk/{util => }/state/package-info.java | 2 +- .../apache/beam/sdk/testing/UsesMapState.java | 2 +- .../apache/beam/sdk/testing/UsesSetState.java | 2 +- .../org/apache/beam/sdk/transforms/DoFn.java | 4 ++-- .../beam/sdk/transforms/DoFnTester.java | 2 +- .../beam/sdk/transforms/GroupIntoBatches.java | 10 ++++----- .../org/apache/beam/sdk/transforms/ParDo.java | 2 +- .../sdk/transforms/reflect/DoFnInvoker.java | 2 +- .../sdk/transforms/reflect/DoFnSignature.java | 4 ++-- .../transforms/reflect/DoFnSignatures.java | 4 ++-- .../beam/sdk/util/CombineContextFactory.java | 2 +- .../apache/beam/sdk/util/CombineFnUtil.java | 2 +- .../apache/beam/sdk/transforms/ParDoTest.java | 14 ++++++------ .../transforms/reflect/DoFnInvokersTest.java | 6 ++--- .../reflect/DoFnSignaturesTest.java | 8 +++---- .../beam/sdk/util/CombineFnUtilTest.java | 2 +- 84 files changed, 252 insertions(+), 255 deletions(-) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/BagState.java (96%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/CombiningState.java (98%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/GroupingState.java (97%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/MapState.java (98%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/ReadableState.java (97%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/ReadableStates.java (97%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/SetState.java (97%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/State.java (96%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/StateBinder.java (98%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/StateContext.java (97%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/StateContexts.java (97%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/StateSpec.java (98%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/StateSpecs.java (99%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/ValueState.java (96%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/WatermarkHoldState.java (97%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => }/state/package-info.java (95%) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java index 430056718696..eeea6d144474 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternals.java @@ -41,22 +41,22 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; -import org.apache.beam.sdk.util.state.StateContexts; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.joda.time.Instant; /** diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/StateInternalsProxy.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/StateInternalsProxy.java index 746be2f8f572..ccf7e43e536c 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/StateInternalsProxy.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/StateInternalsProxy.java @@ -24,8 +24,8 @@ import org.apache.beam.runners.core.StateInternalsFactory; import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateTag; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; /** * State internals for reusable processing context. diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java index 8b48a74148b8..a7e64af4dd5b 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/ApexStateInternalsTest.java @@ -33,16 +33,16 @@ import org.apache.beam.runners.core.StateTags; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.GroupingState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.GroupingState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java index bb1b1cd63184..278c12bf8bcf 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java @@ -37,6 +37,9 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.runners.PTransformMatcher; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; @@ -60,9 +63,6 @@ import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java index fd1772ad2761..849400f1d06d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.beam.runners.core.OldDoFn.Context; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.FinishBundleContext; import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; @@ -34,7 +35,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Duration; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java index 199ce41a867d..59814bc754c2 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java @@ -30,22 +30,22 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.ReadableStates; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.ReadableStates; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; -import org.apache.beam.sdk.util.state.StateContexts; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java index 2faedbb0f630..185892eca81b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java @@ -36,9 +36,9 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.MapCoder; import org.apache.beam.sdk.coders.SetCoder; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.state.ValueState; /** * An {@link ActiveWindowSet} for merging {@link WindowFn} implementations. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingStateAccessor.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingStateAccessor.java index 5ffb9a22218a..361e8c527d3d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingStateAccessor.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingStateAccessor.java @@ -20,8 +20,8 @@ import java.util.Map; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.state.State; /** * Interface for accessing persistent state while windows are merging. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java index 06dcc9cdcfe1..8df7ad005aa4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java @@ -18,12 +18,12 @@ package org.apache.beam.runners.core; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.ReadableState; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.ReadableState; /** * Tracks which windows have non-empty panes. Specifically, which windows have new elements since diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index a51041ed9a99..8e80a694d437 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -27,6 +27,7 @@ import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.FinishBundleContext; import org.apache.beam.sdk.transforms.DoFn.StartBundleContext; @@ -37,7 +38,6 @@ import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Duration; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java index 66b39608a4af..24a4576204fd 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java @@ -20,13 +20,13 @@ import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.ValueState; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java index cd62d859aa54..bcc47a84600b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java @@ -18,11 +18,11 @@ package org.apache.beam.runners.core; import java.io.Serializable; +import org.apache.beam.sdk.state.ReadableState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.Timers; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.ReadableState; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index 3031ebf6d8f7..cb1a15964e32 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -28,15 +28,15 @@ import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timers; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; -import org.apache.beam.sdk.util.state.StateContexts; import org.apache.beam.sdk.values.PCollectionView; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java index b29f9d0281bb..af750104e6dc 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java @@ -28,12 +28,12 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SetCoder; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.PCollectionView; /** 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 768378daceca..200a43b5952f 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 @@ -32,6 +32,8 @@ 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; +import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.FinishBundleContext; import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; @@ -55,8 +57,6 @@ import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Duration; 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/SplittableParDo.java index 7cf5e88a4c99..1fda7d92a176 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/SplittableParDo.java @@ -29,6 +29,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptions; +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; @@ -47,8 +49,6 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateAccessor.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateAccessor.java index eda896bc683f..6c4052023586 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateAccessor.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateAccessor.java @@ -19,7 +19,7 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.util.state.State; +import org.apache.beam.sdk.state.State; /** * Interface for accessing a {@link StateTag} in the current context. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateInternals.java index c2e94120c707..abf961dea8a4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateInternals.java @@ -19,9 +19,9 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; /** * {@code StateInternals} describes the functionality a runner needs to provide for the diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java index f6b91032e5d7..8285c7245d25 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java @@ -23,14 +23,14 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.GroupingState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.GroupingState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTable.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTable.java index 1bf4ff511392..d996729a476a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTable.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTable.java @@ -22,8 +22,8 @@ import java.util.Map; import java.util.Set; import org.apache.beam.runners.core.StateTag.StateBinder; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; /** * Table mapping {@code StateNamespace} and {@code StateTag} to a {@code State} instance. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java index 38e9dea6f712..f7c309e208a1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTag.java @@ -22,18 +22,18 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; /** * An address and specification for a persistent state cell. This includes a unique identifier for diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java index ca8b2387f52e..53f9edcc9f4b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java @@ -25,20 +25,20 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateBinder; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateBinder; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; /** * Static utility methods for creating {@link StateTag} instances. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java index e3717a89daeb..0a7701aed8d9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java @@ -21,6 +21,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; @@ -31,8 +33,6 @@ import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateSpec; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java index f18460ae4044..c189b0d7ee3f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java @@ -19,15 +19,15 @@ import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.GroupingState; +import org.apache.beam.sdk.state.ReadableState; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.GroupingState; -import org.apache.beam.sdk.util.state.ReadableState; /** * {@link ReduceFn} implementing the default reduction behaviors of {@link GroupByKey}. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TestInMemoryStateInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TestInMemoryStateInternals.java index 18b50dba0b39..2052c039f80a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TestInMemoryStateInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TestInMemoryStateInternals.java @@ -20,8 +20,8 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.WatermarkHoldState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java index e6e4ffb3bafe..ce15244d795c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java @@ -22,14 +22,14 @@ import com.google.common.annotations.VisibleForTesting; import java.io.Serializable; import javax.annotation.Nullable; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java index ed2c26f68a54..b972985fa3ac 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java @@ -30,12 +30,12 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.GroupingState; import org.apache.beam.sdk.transforms.Combine.Holder; import org.apache.beam.sdk.transforms.Min; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.GroupingState; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java index 52fb5ff495e3..b9fbac34d0fb 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachine.java @@ -26,8 +26,8 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.state.CombiningState; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.state.CombiningState; /** * {@link TriggerStateMachine}s that fire based on properties of the elements in the current pane. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java index a0569374b396..84b0453d12fc 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java @@ -34,11 +34,11 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachine.MergingTriggerInfo; import org.apache.beam.runners.core.triggers.TriggerStateMachine.TriggerInfo; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timers; -import org.apache.beam.sdk.util.state.State; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java index fc2f69681480..324d44d7d515 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java @@ -29,9 +29,9 @@ import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.core.StateTags; import org.apache.beam.sdk.coders.BitSetCoder; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.Timers; -import org.apache.beam.sdk.util.state.ValueState; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryStateInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryStateInternalsTest.java index 16f7f26dc965..b526305cea00 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryStateInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryStateInternalsTest.java @@ -32,18 +32,18 @@ import java.util.Objects; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.GroupingState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.GroupingState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; 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 d4ff49e0b849..d7d8261cca5f 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 @@ -30,6 +30,9 @@ import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.metrics.MetricsEnvironment; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; @@ -38,9 +41,6 @@ import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Duration; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java index d2af93cb5a32..3c701c77695a 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java @@ -39,20 +39,20 @@ import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.core.StateTag.StateBinder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; -import org.apache.beam.sdk.util.state.StateContexts; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.joda.time.Instant; /** 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 f278e08ba98f..ca302b8b11a6 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 @@ -39,6 +39,7 @@ import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; @@ -48,7 +49,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java index 3e29a69bdf82..46f26a1d4701 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java @@ -38,18 +38,18 @@ import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.GroupingState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.GroupingState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; 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 40582d9805d3..3eff11a3436a 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 @@ -43,6 +43,7 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; @@ -56,7 +57,6 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; 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 67f2fd5c41d8..95bbc36ac145 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 @@ -44,6 +44,9 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; @@ -58,9 +61,6 @@ import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java index b4d4b08dfc20..5b7bff07fc7a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.flink; import java.io.IOException; - import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.metrics.MetricResults; import org.joda.time.Duration; 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 41db61edbb49..c3b979401ff1 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 @@ -18,9 +18,7 @@ package org.apache.beam.runners.flink.translation.types; import java.io.IOException; - import org.apache.beam.sdk.coders.Coder; - import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; 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 518d6bed308d..e44ad57b7a8f 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 @@ -57,6 +57,7 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.state.KeyGroupCheckpointedOperator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -67,7 +68,6 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.flink.core.memory.DataInputViewStreamWrapper; 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 cfe3f9bf66eb..f44e66801e4a 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 @@ -32,20 +32,20 @@ import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.MapCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CombineContextFactory; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; -import org.apache.beam.sdk.util.state.StateContexts; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java index c9b7797ef1a8..d6af4f9be944 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkKeyGroupStateInternals.java @@ -35,20 +35,20 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; -import org.apache.beam.sdk.util.state.StateContexts; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.KeyGroupsList; import org.apache.flink.runtime.state.KeyedStateBackend; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java index 3d38f88c193a..bb2a9fff45d9 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java @@ -24,19 +24,19 @@ import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; -import org.apache.beam.sdk.util.state.StateContexts; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.runtime.state.OperatorStateBackend; 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 cea6e0f719fb..9cb742ee7e30 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 @@ -29,22 +29,22 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.CombineContextFactory; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; -import org.apache.beam.sdk.util.state.StateContexts; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.base.StringSerializer; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java index bda30e4b8be7..2a51be619c5c 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java @@ -39,6 +39,9 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.testing.PCollectionViewTesting; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; @@ -52,9 +55,6 @@ import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java index eb2c05f68703..2b96d91e917a 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java @@ -31,12 +31,12 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkBroadcastStateInternals; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.GroupingState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.GroupingState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java index 0e0267bf9cac..40123737d2ae 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkKeyGroupStateInternalsTest.java @@ -34,9 +34,9 @@ import org.apache.beam.runners.core.StateTags; import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkKeyGroupStateInternals; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.ReadableState; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.ReadableState; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.typeutils.GenericTypeInfo; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.java index 8033a9daf47e..17cd3f5d2c84 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkSplitStateInternalsTest.java @@ -27,8 +27,8 @@ import org.apache.beam.runners.core.StateTags; import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkSplitStateInternals; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.ReadableState; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.ReadableState; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java index cd00d9edf234..35d2b786b82d 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java @@ -32,17 +32,17 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.GroupingState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.GroupingState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.typeutils.GenericTypeInfo; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java index 38129ab08216..d2ab3579f75a 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java @@ -41,14 +41,14 @@ import org.apache.beam.sdk.extensions.gcp.auth.TestCredential; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.TransformHierarchy.Node; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; 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.util.GcsUtil; import org.apache.beam.sdk.util.gcsfs.GcsPath; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index 70f00fb0f2a0..986ed5c757a1 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -85,6 +85,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; @@ -97,9 +100,6 @@ import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.gcsfs.GcsPath; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java index afaba3a571f0..5ea0b0331f3c 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java @@ -30,20 +30,20 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.ReadableState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateContext; -import org.apache.beam.sdk.util.state.StateContexts; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.joda.time.Instant; 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 28bbc3cca40a..1db0e86d6afc 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 @@ -373,7 +373,7 @@ - + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/BagState.java similarity index 96% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/BagState.java index e0eebe598dff..189d1517a302 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/BagState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/BagState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; /** * State containing a bag values. Items can be added to the bag and the contents read out. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/CombiningState.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/CombiningState.java index 80e4dc9478f3..6080127d3800 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CombiningState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/CombiningState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import org.apache.beam.sdk.transforms.Combine.CombineFn; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/GroupingState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/GroupingState.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/GroupingState.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/GroupingState.java index bd7a8d9cf9d6..3a12e79ac4a3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/GroupingState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/GroupingState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import org.apache.beam.sdk.transforms.Combine.CombineFn; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MapState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/MapState.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MapState.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/MapState.java index fb7e807804d0..9f0eee9a59cf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MapState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/MapState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import java.util.Map; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableState.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableState.java index c3e9936b9305..b29ab266a1ca 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableStates.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableStates.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java index 819eda694a0c..d8df04e8e255 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ReadableStates.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/SetState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/SetState.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/SetState.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/SetState.java index 56ea510411be..14aa6406f4fc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/SetState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/SetState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; /** * State containing no duplicate elements. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/State.java similarity index 96% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/State.java index 3a49f017bec7..6b10c9100492 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/State.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/State.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; /** * Base interface for all state locations. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateBinder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateBinder.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateBinder.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateBinder.java index 48fa7428dd28..ee4aa78983b6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateBinder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateBinder.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Combine; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContext.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java index 887a5f11db1a..110a51569065 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContexts.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContexts.java index 2ce959430569..63afe4f2aada 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContexts.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpec.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpec.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpec.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpec.java index 8eda218193c8..3b0b8407ca97 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpec.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpec.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import java.io.Serializable; import org.apache.beam.sdk.annotations.Experimental; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java similarity index 99% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java index 49d57228eb39..09cc4e733bf4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateSpecs.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import static com.google.common.base.Preconditions.checkArgument; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ValueState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ValueState.java similarity index 96% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ValueState.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/ValueState.java index b4322039a6c6..ca97db2fae4b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/ValueState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ValueState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/WatermarkHoldState.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/WatermarkHoldState.java index ae9b700f1fa2..9f6c20383747 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/WatermarkHoldState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/package-info.java similarity index 95% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/package-info.java index b9bec1600d01..de5eeeb22468 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/package-info.java @@ -19,4 +19,4 @@ /** * Defines internal utilities for interacting with pipeline state. */ -package org.apache.beam.sdk.util.state; +package org.apache.beam.sdk.state; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMapState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMapState.java index 9bced418d225..585d8b7e481a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMapState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesMapState.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.util.state.MapState; +import org.apache.beam.sdk.state.MapState; /** * Category tag for validation tests which utilize {@link MapState}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSetState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSetState.java index 6fd74bdf4f13..7d82d227a3c9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSetState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSetState.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.util.state.SetState; +import org.apache.beam.sdk.state.SetState; /** * Category tag for validation tests which utilize {@link SetState}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index f3d178e16f84..c85893653093 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -27,6 +27,8 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker; @@ -37,8 +39,6 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index ca7427c9bc64..ead256948c86 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -320,7 +320,7 @@ public RestrictionTracker restrictionTracker() { } @Override - public org.apache.beam.sdk.util.state.State state(String stateId) { + public org.apache.beam.sdk.state.State state(String stateId) { throw new UnsupportedOperationException("DoFnTester doesn't support state yet"); } 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 e1321151ada0..682897965a49 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 @@ -23,16 +23,16 @@ import com.google.common.collect.Iterables; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; +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.ValueState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.TimerSpecs; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index 7dd2cdd53ce8..c45311a7b0ab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -33,6 +33,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.transforms.DoFn.WindowedContext; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; @@ -46,7 +47,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.NameUtils; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java index 3c44afe7b98f..d5a1a9473d10 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java @@ -19,6 +19,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.FinishBundle; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; @@ -28,7 +29,6 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.state.State; /** * Interface for invoking the {@code DoFn} processing methods. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java index 3219f9680a55..72ad4b0ef1ce 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java @@ -27,6 +27,8 @@ import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.StateId; import org.apache.beam.sdk.transforms.DoFn.TimerId; @@ -38,8 +40,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java index bac3befc36df..3dfca8c456ae 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -42,6 +42,8 @@ import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.StateId; import org.apache.beam.sdk.transforms.DoFn.TimerId; @@ -58,8 +60,6 @@ import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.sdk.util.state.State; -import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeParameter; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java index 31d1f645fd7f..f93cb0b50200 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java @@ -18,9 +18,9 @@ package org.apache.beam.sdk.util; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.state.StateContext; import org.apache.beam.sdk.transforms.CombineWithContext.Context; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.state.StateContext; import org.apache.beam.sdk.values.PCollectionView; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java index a3941803c2f0..1b1c35285877 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java @@ -23,12 +23,12 @@ import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.state.StateContext; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.CombineWithContext.Context; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.state.StateContext; /** * Static utility methods that create combine function instances. 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 56051a69689d..1d4192381b23 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 @@ -64,6 +64,13 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -92,13 +99,6 @@ import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; -import org.apache.beam.sdk.util.state.BagState; -import org.apache.beam.sdk.util.state.CombiningState; -import org.apache.beam.sdk.util.state.MapState; -import org.apache.beam.sdk.util.state.SetState; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index fe96e87951a4..13e46d517ec1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -38,6 +38,9 @@ import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker.FakeArgumentProvider; import org.apache.beam.sdk.transforms.reflect.testhelper.DoFnInvokersTestHelper; @@ -50,9 +53,6 @@ import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java index f099d5d1b846..27e0b89626be 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java @@ -29,6 +29,10 @@ 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.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +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.reflect.DoFnSignature.Parameter; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.ProcessContextParameter; @@ -41,10 +45,6 @@ import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.TimerSpecs; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; -import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptor; import org.hamcrest.Matcher; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java index 798e8dca4663..d16671b81b68 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java @@ -27,10 +27,10 @@ import java.io.NotSerializableException; import java.io.ObjectOutputStream; import java.util.List; +import org.apache.beam.sdk.state.StateContexts; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.CombineWithContext.Context; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.state.StateContexts; import org.junit.Before; import org.junit.Rule; import org.junit.Test; From ac01ec7afb43ae0bc4198234d1de16a830f95b10 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 2 May 2017 10:41:01 -0700 Subject: [PATCH 051/387] Add @Internal and @Experimental to state package --- .../org/apache/beam/sdk/state/BagState.java | 4 + .../apache/beam/sdk/state/CombiningState.java | 3 + .../apache/beam/sdk/state/GroupingState.java | 3 + .../org/apache/beam/sdk/state/MapState.java | 3 + .../apache/beam/sdk/state/ReadableStates.java | 7 +- .../org/apache/beam/sdk/state/SetState.java | 4 + .../java/org/apache/beam/sdk/state/State.java | 4 + .../apache/beam/sdk/state/StateBinder.java | 7 +- .../apache/beam/sdk/state/StateContext.java | 5 +- .../apache/beam/sdk/state/StateContexts.java | 5 +- .../org/apache/beam/sdk/state/StateSpec.java | 18 +++- .../org/apache/beam/sdk/state/StateSpecs.java | 88 +++++++++++-------- .../beam/sdk/state/WatermarkHoldState.java | 11 ++- .../apache/beam/sdk/state/package-info.java | 2 +- 14 files changed, 109 insertions(+), 55 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/BagState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/BagState.java index 189d1517a302..10fba05d1fbb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/BagState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/BagState.java @@ -17,11 +17,15 @@ */ package org.apache.beam.sdk.state; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; + /** * State containing a bag values. Items can be added to the bag and the contents read out. * * @param The type of elements in the bag. */ +@Experimental(Kind.STATE) public interface BagState extends GroupingState> { @Override BagState readLater(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/CombiningState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/CombiningState.java index 6080127d3800..ddda25519e3f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/CombiningState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/CombiningState.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.state; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.transforms.Combine.CombineFn; /** @@ -27,6 +29,7 @@ * @param the type of accumulator * @param the type of value extracted from the state */ +@Experimental(Kind.STATE) public interface CombiningState extends GroupingState { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/GroupingState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/GroupingState.java index 3a12e79ac4a3..d99ff25515cf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/GroupingState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/GroupingState.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.state; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.transforms.Combine.CombineFn; /** @@ -26,6 +28,7 @@ * @param the type of values added to the state * @param the type of value extracted from the state */ +@Experimental(Kind.STATE) public interface GroupingState extends ReadableState, State { /** * Add a value to the buffer. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/MapState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/MapState.java index 9f0eee9a59cf..649c3c7653d8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/MapState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/MapState.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.state; import java.util.Map; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; /** * An object that maps keys to values. @@ -27,6 +29,7 @@ * @param the type of keys maintained by this map * @param the type of mapped values */ +@Experimental(Kind.STATE) public interface MapState extends State { /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java index d8df04e8e255..6977a9718bb3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/ReadableStates.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.state; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.annotations.Internal; /** - * Utilities for constructing and manipulating {@link ReadableState} instances. + * For internal use only; no backwards-compatibility guarantees. */ -@Experimental(Kind.STATE) +@Internal public class ReadableStates { /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/SetState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/SetState.java index 14aa6406f4fc..cb9a0e6dd780 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/SetState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/SetState.java @@ -17,12 +17,16 @@ */ package org.apache.beam.sdk.state; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; + /** * State containing no duplicate elements. * Items can be added to the set and the contents read out. * * @param The type of elements in the set. */ +@Experimental(Kind.STATE) public interface SetState extends GroupingState> { /** * Returns true if this set contains the specified element. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/State.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/State.java index 6b10c9100492..0c0ca32f2ecc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/State.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/State.java @@ -17,12 +17,16 @@ */ package org.apache.beam.sdk.state; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; + /** * Base interface for all state locations. * *

    Specific types of state add appropriate accessors for reading and writing values, see * {@link ValueState}, {@link BagState}, and {@link GroupingState}. */ +@Experimental(Kind.STATE) public interface State { /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateBinder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateBinder.java index ee4aa78983b6..af834c3cdc91 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateBinder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateBinder.java @@ -17,14 +17,19 @@ */ package org.apache.beam.sdk.state; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; /** - * Visitor for binding a {@link StateSpec} and to the associated {@link State}. + * For internal use only; no backwards-compatibility guarantees. + * For internal use only; no backwards-compatibility guarantees. + * + *

    Visitor for binding a {@link StateSpec} and to the associated {@link State}. */ +@Internal public interface StateBinder { ValueState bindValue( String id, StateSpec> spec, Coder coder); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java index 110a51569065..52177cc0a41c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java @@ -17,13 +17,16 @@ */ package org.apache.beam.sdk.state; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollectionView; /** - * Information accessible the state API. + * For internal use only; no backwards-compatibility guarantees. + * For internal use only; no backwards-compatibility guarantees. */ +@Internal public interface StateContext { /** * Returns the {@code PipelineOptions} specified with the diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContexts.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContexts.java index 63afe4f2aada..2e21a27c1a6a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContexts.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContexts.java @@ -17,13 +17,16 @@ */ package org.apache.beam.sdk.state; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollectionView; /** - * Factory that produces {@link StateContext} based on different inputs. + * For internal use only; no backwards-compatibility guarantees. + * For internal use only; no backwards-compatibility guarantees. */ +@Internal public class StateContexts { private static final StateContext NULL_CONTEXT = new StateContext() { 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 3b0b8407ca97..b0412bf49c6d 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 @@ -20,6 +20,7 @@ import java.io.Serializable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.Coder; /** @@ -32,22 +33,31 @@ public interface StateSpec extends Serializable { /** - * Use the {@code binder} to create an instance of {@code StateT} appropriate for this address. + * For internal use only; no backwards-compatibility guarantees. + * + *

    Use the {@code binder} to create an instance of {@code StateT} appropriate for this address. */ + @Internal StateT bind(String id, StateBinder binder); /** - * Given {code coders} are inferred from type arguments defined for this class. Coders which are - * already set should take precedence over offered coders. + * For internal use only; no backwards-compatibility guarantees. + * + *

    Given {code coders} are inferred from type arguments defined for this class. Coders which + * are already set should take precedence over offered coders. * * @param coders Array of coders indexed by the type arguments order. Entries might be null if the * coder could not be inferred. */ + @Internal void offerCoders(Coder[] coders); /** - * Validates that this {@link StateSpec} has been specified correctly and finalizes it. + * For internal use only; no backwards-compatibility guarantees. + * + *

    Validates that this {@link StateSpec} has been specified correctly and finalizes it. * Automatically invoked when the pipeline is built. */ + @Internal void finishSpecifying(); } 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 09cc4e733bf4..8a3c87e4c6a4 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 @@ -23,6 +23,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -94,42 +95,6 @@ StateSpec> combining( return combiningInternal(accumCoder, combineFn); } - /** - * Create a state spec for values that use a {@link CombineFn} to automatically merge multiple - * {@code InputT}s into a single {@code OutputT}. - * - *

    This determines the {@code Coder} from the given {@code Coder}, and should - * only be used to initialize static values. - */ - public static - StateSpec> - combiningFromInputInternal( - Coder inputCoder, CombineFn combineFn) { - try { - Coder accumCoder = combineFn.getAccumulatorCoder(STANDARD_REGISTRY, inputCoder); - return combiningInternal(accumCoder, combineFn); - } catch (CannotProvideCoderException e) { - throw new IllegalArgumentException( - "Unable to determine accumulator coder for " - + combineFn.getClass().getSimpleName() - + " from " - + inputCoder, - e); - } - } - - private static - StateSpec> combiningInternal( - Coder accumCoder, CombineFn combineFn) { - return new CombiningStateSpec(accumCoder, combineFn); - } - - private static - StateSpec> combiningInternal( - Coder accumCoder, CombineFnWithContext combineFn) { - return new CombiningWithContextStateSpec(accumCoder, combineFn); - } - /** * Create a state spec that is optimized for adding values frequently, and occasionally retrieving * all the values that have been added. @@ -172,13 +137,62 @@ public static StateSpec> map(Coder keyCoder, Coder v return new MapStateSpec<>(keyCoder, valueCoder); } - /** Create a state spec for holding the watermark. */ + /** + * For internal use only; no backwards-compatibility guarantees. + * + *

    Create a state spec for values that use a {@link CombineFn} to automatically merge multiple + * {@code InputT}s into a single {@code OutputT}. + * + *

    This determines the {@code Coder} from the given {@code Coder}, and should + * only be used to initialize static values. + */ + @Internal + public static + StateSpec> combiningFromInputInternal( + Coder inputCoder, CombineFn combineFn) { + try { + Coder accumCoder = combineFn.getAccumulatorCoder(STANDARD_REGISTRY, inputCoder); + return combiningInternal(accumCoder, combineFn); + } catch (CannotProvideCoderException e) { + throw new IllegalArgumentException( + "Unable to determine accumulator coder for " + + combineFn.getClass().getSimpleName() + + " from " + + inputCoder, + e); + } + } + + private static + StateSpec> combiningInternal( + Coder accumCoder, CombineFn combineFn) { + return new CombiningStateSpec(accumCoder, combineFn); + } + + private static + StateSpec> combiningInternal( + Coder accumCoder, CombineFnWithContext combineFn) { + return new CombiningWithContextStateSpec(accumCoder, combineFn); + } + + /** + * For internal use only; no backwards-compatibility guarantees. + * + *

    Create a state spec for a watermark hold. + */ + @Internal public static StateSpec watermarkStateInternal( TimestampCombiner timestampCombiner) { return new WatermarkStateSpecInternal(timestampCombiner); } + /** + * For internal use only; no backwards-compatibility guarantees. + * + *

    Convert a combining state spec to a bag of accumulators. + */ + @Internal public static StateSpec> convertToBagSpecInternal( StateSpec> combiningSpec) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/WatermarkHoldState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/WatermarkHoldState.java index 9f6c20383747..38e2cbc27b64 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/WatermarkHoldState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/WatermarkHoldState.java @@ -17,18 +17,17 @@ */ package org.apache.beam.sdk.state; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.joda.time.Instant; /** - * A {@link State} accepting and aggregating output timestamps, which determines the time to which - * the output watermark must be held. + * For internal use only; no backwards-compatibility guarantees. * - *

    For internal use only. This API may change at any time. + *

    A {@link State} accepting and aggregating output timestamps, which determines the time to + * which the output watermark must be held. */ -@Experimental(Kind.STATE) +@Internal public interface WatermarkHoldState extends GroupingState { /** * Return the {@link TimestampCombiner} which will be used to determine a watermark hold time diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/package-info.java index de5eeeb22468..d8b8e9266c2e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/package-info.java @@ -17,6 +17,6 @@ */ /** - * Defines internal utilities for interacting with pipeline state. + * Classes and interfaces for interacting with state. */ package org.apache.beam.sdk.state; From ca41af8fe4711ab4a81c2a33746a64e64fb0ca37 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 2 May 2017 12:46:46 -0700 Subject: [PATCH 052/387] Move user-facing timer-related classes out of util --- .../translation/operators/ApexGroupByKeyOperator.java | 2 +- .../apex/translation/operators/ApexTimerInternals.java | 2 +- .../translation/operators/ApexTimerInternalsTest.java | 2 +- .../apache/beam/runners/core/construction/Triggers.java | 2 +- .../runners/core/construction/PTransformMatchersTest.java | 8 ++++---- .../java/org/apache/beam/runners/core/DoFnAdapters.java | 2 +- .../java/org/apache/beam/runners/core/DoFnRunner.java | 2 +- .../apache/beam/runners/core/InMemoryTimerInternals.java | 2 +- .../beam/runners/core/LateDataDroppingDoFnRunner.java | 2 +- ...tputAndTimeBoundedSplittableProcessElementInvoker.java | 2 +- .../org/apache/beam/runners/core/ProcessFnRunner.java | 2 +- .../beam/runners/core/PushbackSideInputDoFnRunner.java | 2 +- .../main/java/org/apache/beam/runners/core/ReduceFn.java | 2 +- .../apache/beam/runners/core/ReduceFnContextFactory.java | 4 ++-- .../java/org/apache/beam/runners/core/ReduceFnRunner.java | 2 +- .../org/apache/beam/runners/core/SimpleDoFnRunner.java | 6 +++--- .../org/apache/beam/runners/core/SimpleOldDoFnRunner.java | 2 +- .../runners/core/SimplePushbackSideInputDoFnRunner.java | 2 +- .../org/apache/beam/runners/core/SplittableParDo.java | 2 +- .../org/apache/beam/runners/core/StatefulDoFnRunner.java | 2 +- .../java/org/apache/beam/runners/core/TimerInternals.java | 2 +- .../triggers/AfterDelayFromFirstElementStateMachine.java | 2 +- .../core/triggers/AfterProcessingTimeStateMachine.java | 2 +- .../AfterSynchronizedProcessingTimeStateMachine.java | 2 +- .../runners/core/triggers/AfterWatermarkStateMachine.java | 2 +- .../runners/core/triggers/DefaultTriggerStateMachine.java | 2 +- .../beam/runners/core/triggers/TriggerStateMachine.java | 2 +- .../core/triggers/TriggerStateMachineContextFactory.java | 4 ++-- .../runners/core/triggers/TriggerStateMachineRunner.java | 2 +- .../beam/runners/core/InMemoryTimerInternalsTest.java | 2 +- .../apache/beam/runners/core/KeyedWorkItemCoderTest.java | 2 +- .../org/apache/beam/runners/core/ReduceFnRunnerTest.java | 2 +- .../java/org/apache/beam/runners/core/ReduceFnTester.java | 2 +- .../apache/beam/runners/core/SimpleDoFnRunnerTest.java | 8 ++++---- .../core/SimplePushbackSideInputDoFnRunnerTest.java | 2 +- .../org/apache/beam/runners/core/TimerInternalsTest.java | 2 +- .../runners/core/triggers/TriggerStateMachineTester.java | 4 ++-- .../runners/core/triggers/TriggerStateMachinesTest.java | 2 +- .../apache/beam/runners/direct/DirectTimerInternals.java | 2 +- .../org/apache/beam/runners/direct/WatermarkManager.java | 2 +- .../beam/runners/direct/DirectTimerInternalsTest.java | 2 +- ...oFnLifecycleManagerRemovingTransformEvaluatorTest.java | 2 +- .../apache/beam/runners/direct/EvaluationContextTest.java | 2 +- .../apache/beam/runners/direct/WatermarkManagerTest.java | 2 +- .../flink/metrics/DoFnRunnerWithMetricsUpdate.java | 2 +- .../translation/wrappers/streaming/DoFnOperator.java | 2 +- .../beam/runners/flink/streaming/DoFnOperatorTest.java | 8 ++++---- .../beam/runners/spark/stateful/SparkTimerInternals.java | 2 +- .../runners/spark/translation/DoFnRunnerWithMetrics.java | 2 +- .../java/org/apache/beam/sdk/options/ValueProviders.java | 1 - .../org/apache/beam/sdk/{util => state}/TimeDomain.java | 2 +- .../java/org/apache/beam/sdk/{util => state}/Timer.java | 2 +- .../org/apache/beam/sdk/{util => state}/TimerSpec.java | 2 +- .../org/apache/beam/sdk/{util => state}/TimerSpecs.java | 4 ++-- .../java/org/apache/beam/sdk/{util => state}/Timers.java | 2 +- .../org/apache/beam/sdk/testing/TestPipelineOptions.java | 2 -- .../main/java/org/apache/beam/sdk/transforms/DoFn.java | 6 +++--- .../java/org/apache/beam/sdk/transforms/DoFnTester.java | 2 +- .../org/apache/beam/sdk/transforms/GroupIntoBatches.java | 8 ++++---- .../main/java/org/apache/beam/sdk/transforms/Regex.java | 1 - .../transforms/reflect/ByteBuddyDoFnInvokerFactory.java | 2 +- .../apache/beam/sdk/transforms/reflect/DoFnInvoker.java | 2 +- .../apache/beam/sdk/transforms/reflect/DoFnSignature.java | 4 ++-- .../beam/sdk/transforms/reflect/DoFnSignatures.java | 4 ++-- .../beam/sdk/transforms/windowing/AfterWatermark.java | 2 +- .../src/main/java/org/apache/beam/sdk/util/ClassPath.java | 1 + .../beam/sdk/io/DrunkWritableByteChannelFactory.java | 1 - .../test/java/org/apache/beam/sdk/io/TFRecordIOTest.java | 2 -- .../java/org/apache/beam/sdk/metrics/GaugeCellTest.java | 1 + .../org/apache/beam/sdk/options/PipelineOptionsTest.java | 1 - .../java/org/apache/beam/sdk/transforms/ParDoTest.java | 8 ++++---- .../java/org/apache/beam/sdk/transforms/RegexTest.java | 2 -- .../beam/sdk/transforms/reflect/DoFnInvokersTest.java | 8 ++++---- .../beam/sdk/transforms/reflect/DoFnSignaturesTest.java | 8 ++++---- .../beam/sdk/transforms/reflect/OnTimerInvokersTest.java | 6 +++--- .../reflect/testhelper/DoFnInvokersTestHelper.java | 6 +++--- 76 files changed, 104 insertions(+), 112 deletions(-) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => state}/TimeDomain.java (97%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => state}/Timer.java (98%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => state}/TimerSpec.java (97%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => state}/TimerSpecs.java (91%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => state}/Timers.java (98%) 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 4fdb60017c8e..6131bdf41ae9 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 @@ -45,11 +45,11 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.NullSideInputReader; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java index 15ccbee1d737..1eb224c266a7 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java @@ -35,8 +35,8 @@ import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.TimeDomain; import org.joda.time.Instant; /** diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java index ee142e2942e8..7b52223e5478 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java @@ -32,8 +32,8 @@ import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.runners.core.TimerInternals.TimerDataCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.joda.time.Instant; import org.junit.Test; 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/Triggers.java index 81f738da0dd5..5e735715b3ee 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/Triggers.java @@ -25,6 +25,7 @@ import java.util.List; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.AfterAll; import org.apache.beam.sdk.transforms.windowing.AfterEach; import org.apache.beam.sdk.transforms.windowing.AfterFirst; @@ -43,7 +44,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.ReshuffleTrigger; -import org.apache.beam.sdk.util.TimeDomain; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java index 278c12bf8bcf..ba935a4e9a0b 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java @@ -39,6 +39,10 @@ import org.apache.beam.sdk.runners.PTransformMatcher; 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.state.ValueState; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -57,10 +61,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.PCollectionViews; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java index 849400f1d06d..af59a40b7518 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnAdapters.java @@ -21,6 +21,7 @@ import org.apache.beam.runners.core.OldDoFn.Context; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.Timer; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.FinishBundleContext; import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; @@ -34,7 +35,6 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Duration; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index b29adcc1c7da..30648f6e5820 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -17,9 +17,9 @@ */ package org.apache.beam.runners.core; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java index f8aaa0904971..e68bb2407c52 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java @@ -27,8 +27,8 @@ import java.util.NavigableSet; import java.util.TreeSet; import javax.annotation.Nullable; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowTracing; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index d6ba8f5ca46e..872ee3e06657 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -23,8 +23,8 @@ import com.google.common.collect.Iterables; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index 8e80a694d437..16bdfa361c0d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -28,6 +28,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.Timer; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.FinishBundleContext; import org.apache.beam.sdk.transforms.DoFn.StartBundleContext; @@ -36,7 +37,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; 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 3ae3f5068c46..7cbf0d214053 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 @@ -25,10 +25,10 @@ import java.util.Collections; import org.apache.beam.runners.core.StateNamespaces.WindowNamespace; import org.apache.beam.runners.core.TimerInternals.TimerData; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java index bab1dc7317fc..8f21086794d9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java @@ -17,9 +17,9 @@ */ package org.apache.beam.runners.core; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java index bcc47a84600b..8e2ff7368b1d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java @@ -19,9 +19,9 @@ import java.io.Serializable; import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.Timers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.Timers; import org.apache.beam.sdk.util.WindowingStrategy; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index cb1a15964e32..550b9b9769b4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -31,11 +31,11 @@ import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.state.StateContext; import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timers; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index d3dc067e31ac..a949eddfb116 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -45,6 +45,7 @@ import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -54,7 +55,6 @@ import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; 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 200a43b5952f..2567bbaa3b57 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 @@ -34,6 +34,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.state.StateSpec; +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.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.FinishBundleContext; import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; @@ -51,9 +54,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java index 4c3149acba5c..7454dc9248d4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java @@ -29,6 +29,7 @@ import org.apache.beam.runners.core.ExecutionContext.StepContext; import org.apache.beam.runners.core.OldDoFn.RequiresWindowAccess; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; @@ -36,7 +37,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java index 50d301bc18e6..36a04d9e2f04 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java @@ -23,9 +23,9 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.joda.time.Instant; 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/SplittableParDo.java index 1fda7d92a176..74ffeb635339 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/SplittableParDo.java @@ -29,6 +29,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptions; +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; @@ -46,7 +47,6 @@ 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.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java index 0a7701aed8d9..346694566084 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java @@ -23,13 +23,13 @@ import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; 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.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java index 375cdf97a257..21fe4301ea57 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java @@ -30,9 +30,9 @@ import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java index b972985fa3ac..8d8d0de40bbe 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java @@ -32,10 +32,10 @@ import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.state.CombiningState; import org.apache.beam.sdk.state.GroupingState; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.Combine.Holder; import org.apache.beam.sdk.transforms.Min; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.TimeDomain; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java index eaf5613cc795..e813d33dd963 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachine.java @@ -21,8 +21,8 @@ import java.util.Objects; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.TimeDomain; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java index 07fab223739c..e1cd8978abb6 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachine.java @@ -21,8 +21,8 @@ import java.util.Collections; import java.util.List; import javax.annotation.Nullable; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.TimeDomain; import org.joda.time.Instant; // This should not really have the superclass https://issues.apache.org/jira/browse/BEAM-1486 diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java index 1b117d2e6923..c9eee15b857f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java @@ -24,7 +24,7 @@ import javax.annotation.Nullable; import org.apache.beam.runners.core.triggers.TriggerStateMachine.OnceTriggerStateMachine; import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.state.TimeDomain; /** * {@code AfterWatermark} triggers fire based on progress of the system watermark. This time is a diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java index be4dd681c603..4ab9fc9dcfc6 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.core.triggers; import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.state.TimeDomain; /** * A trigger that is equivalent to {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}. See diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java index d622ac91131d..6a2cf0c91a20 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java @@ -24,10 +24,10 @@ import javax.annotation.Nullable; import org.apache.beam.runners.core.MergingStateAccessor; import org.apache.beam.runners.core.StateAccessor; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.TimeDomain; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java index 84b0453d12fc..811f30c6b5a9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java @@ -35,10 +35,10 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachine.TriggerInfo; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timers; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java index 324d44d7d515..88ea6efccb7b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineRunner.java @@ -29,9 +29,9 @@ import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.core.StateTags; import org.apache.beam.sdk.coders.BitSetCoder; +import org.apache.beam.sdk.state.Timers; import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.Timers; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java index 022626030204..704942198d3b 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertThat; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.state.TimeDomain; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java index 4d5963492758..4057c25b48b5 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java @@ -21,9 +21,9 @@ import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.joda.time.Instant; import org.junit.Test; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index 7b91151dc1d4..2b400a40e9a5 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -45,6 +45,7 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.CombineWithContext.Context; import org.apache.beam.sdk.transforms.Sum; @@ -70,7 +71,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; 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 573855fd8d52..eea30e912e02 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 @@ -51,6 +51,7 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -64,7 +65,6 @@ import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; 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 b8fc64ea7a55..d793584ca446 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 @@ -33,16 +33,16 @@ import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.sdk.coders.Coder; +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.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.NullSideInputReader; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java index ba3f9263f884..dabc4f013b86 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java @@ -29,6 +29,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.beam.runners.core.TimerInternals.TimerData; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Sum; @@ -39,7 +40,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IdentitySideInputWindowFn; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java index 768c974d217e..af270d9fd075 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java @@ -25,10 +25,10 @@ import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.runners.core.TimerInternals.TimerDataCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java index 1bdcff25fd2a..51f467ae8171 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java @@ -44,12 +44,12 @@ import org.apache.beam.runners.core.StateNamespaces.WindowNamespace; import org.apache.beam.runners.core.TestInMemoryStateInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timers; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachinesTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachinesTest.java index 7ff3478254a7..5158f505101d 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachinesTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachinesTest.java @@ -23,7 +23,7 @@ import static org.junit.Assert.assertThat; import org.apache.beam.sdk.common.runner.v1.RunnerApi; -import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.state.TimeDomain; import org.joda.time.Duration; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java index 67d568da80dd..a099368855ff 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java @@ -23,7 +23,7 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder; import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; -import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.state.TimeDomain; import org.joda.time.Instant; /** 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 b576e00b0beb..21ba734c017c 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 @@ -55,10 +55,10 @@ import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java index c8ac695cb091..f80515fba9f7 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java @@ -28,7 +28,7 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder; import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.state.TimeDomain; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java index 1ac4d6d2773a..d98e51d83a50 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java @@ -29,10 +29,10 @@ import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.TimerInternals.TimerData; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.hamcrest.Matchers; import org.joda.time.Instant; 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 3eff11a3436a..63864f0856e2 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 @@ -44,6 +44,7 @@ import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; @@ -54,7 +55,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index e6506a812221..e1e6ab59cdd4 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -43,6 +43,7 @@ import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; @@ -55,7 +56,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java index 29a1a5248682..dae91fe007fe 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java @@ -21,8 +21,8 @@ import java.io.IOException; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.sdk.metrics.MetricsEnvironment; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.flink.api.common.functions.RuntimeContext; import org.joda.time.Instant; 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 e44ad57b7a8f..5287b85d0bd6 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 @@ -58,6 +58,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -65,7 +66,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java index 2a51be619c5c..80dfa07288c4 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java @@ -41,6 +41,10 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; 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.state.ValueState; import org.apache.beam.sdk.testing.PCollectionViewTesting; import org.apache.beam.sdk.transforms.DoFn; @@ -49,10 +53,6 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java index 646e2699e9fb..107915f7f43d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java @@ -32,8 +32,8 @@ import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.util.GlobalWatermarkHolder.SparkWatermarks; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.spark.broadcast.Broadcast; import org.joda.time.Instant; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java index fa9a9c2599f0..d74b25373387 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java @@ -24,8 +24,8 @@ import org.apache.beam.runners.spark.metrics.SparkMetricsContainer; import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.metrics.MetricsEnvironment; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.WindowedValue; import org.apache.spark.Accumulator; import org.joda.time.Instant; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java index e2355eeb98fa..1cc46fe4a83f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; - import java.io.IOException; import java.util.Map; import org.apache.beam.sdk.util.common.ReflectHelpers; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeDomain.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimeDomain.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeDomain.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimeDomain.java index b067ab8cda85..e8149158965d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimeDomain.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimeDomain.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.state; /** * {@code TimeDomain} specifies whether an operation is based on diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/Timer.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timer.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/Timer.java index 9727969fcc28..9458906721b5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/Timer.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.state; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.DoFn; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerSpec.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpec.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerSpec.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpec.java index 5b7717b1b5bb..f08f43e5a8e1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerSpec.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpec.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.state; import java.io.Serializable; import org.apache.beam.sdk.annotations.Experimental; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerSpecs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java similarity index 91% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerSpecs.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java index f289ccdc4770..9efac69671c7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerSpecs.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.state; import com.google.auto.value.AutoValue; import org.apache.beam.sdk.annotations.Experimental; @@ -28,7 +28,7 @@ public class TimerSpecs { public static TimerSpec timer(TimeDomain timeDomain) { - return new AutoValue_TimerSpecs_SimpleTimerSpec(timeDomain); + return new org.apache.beam.sdk.util.AutoValue_TimerSpecs_SimpleTimerSpec(timeDomain); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/Timers.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/state/Timers.java index 7415d7a2f99e..dcdaf00f7eb5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/Timers.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.state; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java index e1ad47bfceec..206bc1f343c4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java @@ -18,9 +18,7 @@ package org.apache.beam.sdk.testing; import com.fasterxml.jackson.annotation.JsonIgnore; - import javax.annotation.Nullable; - import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index c85893653093..befba1dd71d7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -29,6 +29,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.state.StateSpec; +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.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker; @@ -36,9 +39,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index ead256948c86..8a03f3c0064a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -33,6 +33,7 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.state.Timer; import org.apache.beam.sdk.transforms.DoFn.OnTimerContext; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; @@ -43,7 +44,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; 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 682897965a49..b02336339ffa 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 @@ -27,12 +27,12 @@ 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.state.ValueState; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java index 690d321c77b6..8913e749fe05 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.regex.Matcher; import java.util.regex.Pattern; - import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java index 2b6d5be1c595..8ae2c65a34ec 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java @@ -63,6 +63,7 @@ import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.state.Timer; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.OnTimerMethod; @@ -77,7 +78,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.WindowParameter; import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; -import org.apache.beam.sdk.util.Timer; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.values.TypeDescriptor; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java index d5a1a9473d10..6fd40523c28c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java @@ -20,6 +20,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.Timer; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.FinishBundle; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; @@ -28,7 +29,6 @@ import org.apache.beam.sdk.transforms.DoFn.TimerId; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.Timer; /** * Interface for invoking the {@code DoFn} processing methods. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java index 72ad4b0ef1ce..0b4bf90071a8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java @@ -29,6 +29,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.state.StateSpec; +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.StateId; import org.apache.beam.sdk.transforms.DoFn.TimerId; @@ -38,8 +40,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.WindowParameter; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java index 3dfca8c456ae..bb191b124fdf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -44,6 +44,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.state.StateSpec; +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.StateId; import org.apache.beam.sdk.transforms.DoFn.TimerId; @@ -57,8 +59,6 @@ import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java index 14a8c98a7787..5648895f1c5b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java @@ -24,8 +24,8 @@ import java.util.Objects; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; -import org.apache.beam.sdk.util.TimeDomain; import org.joda.time.Instant; /** 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 35632ed063bf..271bce0fed58 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 @@ -19,6 +19,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; + import com.google.common.annotations.Beta; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.CharMatcher; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DrunkWritableByteChannelFactory.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DrunkWritableByteChannelFactory.java index 79f09961db2a..6615a2e9d333 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DrunkWritableByteChannelFactory.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DrunkWritableByteChannelFactory.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; - import org.apache.beam.sdk.io.FileBasedSink.WritableByteChannelFactory; import org.apache.beam.sdk.util.MimeTypes; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java index 9a9e840801c7..d564d3b2d847 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java @@ -30,7 +30,6 @@ import com.google.common.collect.Lists; import com.google.common.io.BaseEncoding; import com.google.common.io.ByteStreams; - import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; @@ -43,7 +42,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; - import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/GaugeCellTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/GaugeCellTest.java index d8ef92825be7..7dbfc4ba822b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/GaugeCellTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/GaugeCellTest.java @@ -19,6 +19,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; + import org.junit.Test; /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java index a13aa011b2ff..5e3211fc1dec 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsTest.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.common.collect.ImmutableMap; - import java.util.HashSet; import java.util.List; import java.util.Map; 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 1d4192381b23..ffdf3d0506a8 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 @@ -70,6 +70,10 @@ import org.apache.beam.sdk.state.SetState; 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.state.ValueState; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -94,10 +98,6 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java index ceebccad9887..969c4ea25c9c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RegexTest.java @@ -20,13 +20,11 @@ import java.io.Serializable; import java.util.Arrays; import java.util.List; - import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index 13e46d517ec1..19b60925c361 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -40,6 +40,10 @@ import org.apache.beam.sdk.coders.VarIntCoder; 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.state.ValueState; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker.FakeArgumentProvider; @@ -48,10 +52,6 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.util.UserCodeException; import org.joda.time.Instant; import org.junit.Before; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java index 27e0b89626be..5b97a553c28d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java @@ -31,6 +31,10 @@ import org.apache.beam.sdk.coders.VarLongCoder; 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.state.ValueState; import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.DoFn; @@ -41,10 +45,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.WindowParameter; import org.apache.beam.sdk.transforms.reflect.DoFnSignaturesTestUtils.FakeDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.Timer; -import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.TimerSpecs; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptor; import org.hamcrest.Matcher; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokersTest.java index 177f15f39224..d31795216943 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokersTest.java @@ -22,11 +22,11 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.TimerSpecs; import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/testhelper/DoFnInvokersTestHelper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/testhelper/DoFnInvokersTestHelper.java index 95e7c4999542..f9fcc5881335 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/testhelper/DoFnInvokersTestHelper.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/testhelper/DoFnInvokersTestHelper.java @@ -21,12 +21,12 @@ import static org.junit.Assert.assertNull; import static org.mockito.Mockito.verify; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvokersTest; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.TimerSpec; -import org.apache.beam.sdk.util.TimerSpecs; /** * Test helper for {@link DoFnInvokersTest}, which needs to test package-private access to DoFns in From b4c808a045d493e93d83c815dc7520c6b213858a Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 2 May 2017 12:31:02 -0700 Subject: [PATCH 053/387] Mark TimeDomain experimental alongside Timers; improve javadoc --- .../org/apache/beam/sdk/state/TimeDomain.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimeDomain.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimeDomain.java index e8149158965d..0526453067b8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimeDomain.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimeDomain.java @@ -17,27 +17,29 @@ */ package org.apache.beam.sdk.state; +import org.apache.beam.sdk.annotations.Experimental; + /** - * {@code TimeDomain} specifies whether an operation is based on - * timestamps of elements or current "real-world" time as reported while processing. + * {@link TimeDomain} specifies whether an operation is based on timestamps of elements or current + * "real-world" time as reported while processing. */ +@Experimental(Experimental.Kind.TIMERS) public enum TimeDomain { /** - * The {@code EVENT_TIME} domain corresponds to the timestamps on the elements. Time advances - * on the system watermark advances. + * The {@link #EVENT_TIME} domain corresponds to the timestamps on the elements. Time advances on + * the system watermark advances. */ EVENT_TIME, /** - * The {@code PROCESSING_TIME} domain corresponds to the current to the current (system) time. + * The {@link #PROCESSING_TIME} domain corresponds to the current to the current (system) time. * This is advanced during execution of the pipeline. */ PROCESSING_TIME, /** - * Same as the {@code PROCESSING_TIME} domain, except it won't fire a timer set for time - * {@code T} until all timers from earlier stages set for a time earlier than {@code T} have - * fired. + * Same as the {@link #PROCESSING_TIME} domain, except it won't fire a timer set for time t + * until all timers from earlier stages set for a time earlier than t have fired. */ SYNCHRONIZED_PROCESSING_TIME } From b61e5bb2b2e1951c62e7defb752561a0a1af5456 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 2 May 2017 12:53:26 -0700 Subject: [PATCH 054/387] Move WindowingStrategy from util to values WindowingStrategy is a property on PCollection that transform authors regularly mess with. It is part of the public API. --- .../apex/translation/operators/ApexGroupByKeyOperator.java | 2 +- .../runners/apex/translation/operators/ApexParDoOperator.java | 2 +- .../runners/apex/translation/ApexGroupByKeyOperatorTest.java | 2 +- .../beam/runners/apex/translation/ParDoTranslatorTest.java | 2 +- .../apache/beam/runners/core/construction/PCollections.java | 2 +- .../beam/runners/core/construction/PrimitiveCreate.java | 2 +- .../apache/beam/runners/core/construction/SdkComponents.java | 2 +- .../beam/runners/core/construction/WindowingStrategies.java | 4 ++-- .../beam/runners/core/construction/PCollectionsTest.java | 2 +- .../runners/core/construction/PTransformMatchersTest.java | 2 +- .../runners/core/construction/ReplacementOutputsTest.java | 2 +- .../beam/runners/core/construction/SdkComponentsTest.java | 4 ++-- .../runners/core/construction/WindowingStrategiesTest.java | 4 ++-- .../main/java/org/apache/beam/runners/core/DoFnRunners.java | 2 +- .../runners/core/GroupAlsoByWindowViaOutputBufferDoFn.java | 2 +- .../beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java | 2 +- .../runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java | 2 +- .../apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java | 2 +- .../apache/beam/runners/core/LateDataDroppingDoFnRunner.java | 2 +- .../main/java/org/apache/beam/runners/core/LateDataUtils.java | 2 +- .../main/java/org/apache/beam/runners/core/NonEmptyPanes.java | 4 ++-- .../src/main/java/org/apache/beam/runners/core/ReduceFn.java | 2 +- .../org/apache/beam/runners/core/ReduceFnContextFactory.java | 2 +- .../java/org/apache/beam/runners/core/ReduceFnRunner.java | 4 ++-- .../java/org/apache/beam/runners/core/SimpleDoFnRunner.java | 2 +- .../org/apache/beam/runners/core/SimpleOldDoFnRunner.java | 2 +- .../java/org/apache/beam/runners/core/SplittableParDo.java | 2 +- .../java/org/apache/beam/runners/core/StatefulDoFnRunner.java | 2 +- .../main/java/org/apache/beam/runners/core/WatermarkHold.java | 2 +- .../core/GroupAlsoByWindowViaOutputBufferDoFnTest.java | 2 +- .../beam/runners/core/GroupAlsoByWindowsProperties.java | 2 +- .../beam/runners/core/LateDataDroppingDoFnRunnerTest.java | 2 +- .../java/org/apache/beam/runners/core/ReduceFnRunnerTest.java | 4 ++-- .../java/org/apache/beam/runners/core/ReduceFnTester.java | 4 ++-- .../org/apache/beam/runners/core/SideInputHandlerTest.java | 2 +- .../org/apache/beam/runners/core/SimpleDoFnRunnerTest.java | 2 +- .../org/apache/beam/runners/core/SplittableParDoTest.java | 2 +- .../org/apache/beam/runners/core/StatefulDoFnRunnerTest.java | 2 +- .../beam/runners/core/triggers/TriggerStateMachineTester.java | 2 +- .../java/org/apache/beam/runners/direct/DirectGroupByKey.java | 2 +- .../org/apache/beam/runners/direct/EvaluationContext.java | 2 +- .../runners/direct/GroupAlsoByWindowEvaluatorFactory.java | 2 +- .../java/org/apache/beam/runners/direct/ParDoEvaluator.java | 2 +- .../apache/beam/runners/direct/ParDoMultiOverrideFactory.java | 2 +- .../org/apache/beam/runners/direct/SideInputContainer.java | 2 +- .../direct/SplittableProcessElementsEvaluatorFactory.java | 2 +- .../beam/runners/direct/StatefulParDoEvaluatorFactory.java | 2 +- .../beam/runners/direct/TestStreamEvaluatorFactory.java | 2 +- .../apache/beam/runners/direct/WatermarkCallbackExecutor.java | 2 +- .../org/apache/beam/runners/direct/CommittedResultTest.java | 2 +- .../org/apache/beam/runners/direct/EvaluationContextTest.java | 2 +- .../beam/runners/direct/KeyedPValueTrackingVisitorTest.java | 2 +- .../apache/beam/runners/direct/SideInputContainerTest.java | 2 +- .../runners/direct/StatefulParDoEvaluatorFactoryTest.java | 2 +- .../apache/beam/runners/direct/ViewOverrideFactoryTest.java | 2 +- .../beam/runners/direct/WatermarkCallbackExecutorTest.java | 2 +- .../beam/runners/direct/WriteWithShardingFactoryTest.java | 2 +- .../beam/runners/flink/FlinkBatchTransformTranslators.java | 2 +- .../beam/runners/flink/FlinkBatchTranslationContext.java | 2 +- .../runners/flink/FlinkStreamingTransformTranslators.java | 2 +- .../translation/functions/AbstractFlinkCombineRunner.java | 2 +- .../flink/translation/functions/FlinkDoFnFunction.java | 2 +- .../functions/FlinkMergingNonShuffleReduceFunction.java | 2 +- .../translation/functions/FlinkPartialReduceFunction.java | 2 +- .../flink/translation/functions/FlinkReduceFunction.java | 2 +- .../flink/translation/functions/FlinkSideInputReader.java | 2 +- .../translation/functions/FlinkStatefulDoFnFunction.java | 2 +- .../translation/functions/HashingFlinkCombineRunner.java | 2 +- .../translation/functions/SortingFlinkCombineRunner.java | 2 +- .../flink/translation/wrappers/streaming/DoFnOperator.java | 2 +- .../wrappers/streaming/SplittableDoFnOperator.java | 2 +- .../translation/wrappers/streaming/WindowDoFnOperator.java | 2 +- .../org/apache/beam/runners/flink/PipelineOptionsTest.java | 2 +- .../apache/beam/runners/flink/streaming/DoFnOperatorTest.java | 2 +- .../java/org/apache/beam/runners/dataflow/AssignWindows.java | 2 +- .../beam/runners/dataflow/BatchStatefulParDoOverrides.java | 2 +- .../org/apache/beam/runners/dataflow/BatchViewOverrides.java | 2 +- .../beam/runners/dataflow/DataflowPipelineTranslator.java | 2 +- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 2 +- .../beam/runners/dataflow/ReshuffleOverrideFactory.java | 2 +- .../java/org/apache/beam/runners/dataflow/util/DoFnInfo.java | 2 +- .../beam/runners/dataflow/DataflowPipelineTranslatorTest.java | 2 +- .../org/apache/beam/runners/dataflow/DataflowRunnerTest.java | 2 +- .../runners/dataflow/transforms/DataflowGroupByKeyTest.java | 2 +- .../beam/runners/dataflow/transforms/DataflowViewTest.java | 2 +- .../java/org/apache/beam/runners/spark/io/CreateStream.java | 2 +- .../spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java | 2 +- .../beam/runners/spark/translation/GroupCombineFunctions.java | 2 +- .../beam/runners/spark/translation/MultiDoFnFunction.java | 2 +- .../runners/spark/translation/SparkAbstractCombineFn.java | 2 +- .../beam/runners/spark/translation/SparkGlobalCombineFn.java | 2 +- .../translation/SparkGroupAlsoByWindowViaOutputBufferFn.java | 2 +- .../beam/runners/spark/translation/SparkKeyedCombineFn.java | 2 +- .../runners/spark/translation/StorageLevelPTransform.java | 2 +- .../beam/runners/spark/translation/TransformTranslator.java | 2 +- .../beam/runners/spark/translation/TranslationUtils.java | 2 +- .../translation/streaming/StreamingTransformTranslator.java | 2 +- .../runners/spark/util/SinglePrimitiveOutputPTransform.java | 2 +- .../apache/beam/runners/spark/util/SparkSideInputReader.java | 2 +- sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java | 2 +- .../src/main/java/org/apache/beam/sdk/state/TimerSpecs.java | 2 +- .../src/main/java/org/apache/beam/sdk/testing/PAssert.java | 2 +- .../src/main/java/org/apache/beam/sdk/testing/TestStream.java | 2 +- .../src/main/java/org/apache/beam/sdk/transforms/Combine.java | 2 +- .../src/main/java/org/apache/beam/sdk/transforms/Flatten.java | 2 +- .../main/java/org/apache/beam/sdk/transforms/GroupByKey.java | 2 +- .../java/org/apache/beam/sdk/transforms/windowing/Window.java | 4 ++-- .../main/java/org/apache/beam/sdk/util/AppliedCombineFn.java | 1 + .../main/java/org/apache/beam/sdk/util/IdentityWindowFn.java | 1 + .../main/java/org/apache/beam/sdk/util/PCollectionViews.java | 1 + .../src/main/java/org/apache/beam/sdk/util/Reshuffle.java | 1 + .../src/main/java/org/apache/beam/sdk/values/PCollection.java | 1 - .../java/org/apache/beam/sdk/values/PCollectionTuple.java | 1 - .../main/java/org/apache/beam/sdk/values/PCollectionView.java | 1 - .../apache/beam/sdk/{util => values}/WindowingStrategy.java | 2 +- .../org/apache/beam/sdk/runners/TransformHierarchyTest.java | 2 +- .../java/org/apache/beam/sdk/runners/TransformTreeTest.java | 2 +- .../org/apache/beam/sdk/testing/PCollectionViewTesting.java | 2 +- .../java/org/apache/beam/sdk/transforms/DoFnTesterTest.java | 2 +- .../java/org/apache/beam/sdk/transforms/GroupByKeyTest.java | 2 +- .../test/java/org/apache/beam/sdk/transforms/ViewTest.java | 2 +- .../org/apache/beam/sdk/transforms/windowing/WindowTest.java | 4 ++-- .../src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java | 1 + .../java/org/apache/beam/sdk/values/PCollectionTupleTest.java | 1 - .../apache/beam/sdk/extensions/protobuf/ByteStringCoder.java | 1 - .../apache/beam/fn/harness/control/ProcessBundleHandler.java | 2 +- .../beam/fn/harness/control/ProcessBundleHandlerTest.java | 2 +- .../org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java | 2 +- 128 files changed, 132 insertions(+), 132 deletions(-) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => values}/WindowingStrategy.java (99%) 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 6131bdf41ae9..9d4e9a235a6b 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 @@ -51,10 +51,10 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 f7242e753a2d..99ad9647322a 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 @@ -61,10 +61,10 @@ import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java index 4b73114da4fe..e31d83051dc5 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java @@ -38,10 +38,10 @@ 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.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java index 1a5c8be4da5c..736b0e736b60 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ParDoTranslatorTest.java @@ -54,12 +54,12 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowingStrategy; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; 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/PCollections.java index 907e54dd21c1..0f2fcb7ca281 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/PCollections.java @@ -22,9 +22,9 @@ import java.io.IOException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.WindowingStrategy; /** * Utility methods for translating {@link PCollection PCollections} to and from Runner API protos. diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java index 5a2140b63ee4..a791a0e20a03 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java @@ -24,12 +24,12 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Create.Values; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; /** * An implementation of {@link Create} that returns a primitive {@link PCollection}. 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 35af3006d2fe..3e7807221e49 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 @@ -34,8 +34,8 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.NameUtils; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; /** SDK objects that will be represented at some later point within a {@link Components} object. */ class SdkComponents { 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 0c400db1681e..395702f51a3e 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 @@ -35,8 +35,8 @@ import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.joda.time.Duration; /** Utilities for working with {@link WindowingStrategy WindowingStrategies}. */ 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 be3755c19ad3..c177c58cb6d6 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 @@ -48,10 +48,10 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java index ba935a4e9a0b..cb28c3449295 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java @@ -62,7 +62,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; @@ -72,6 +71,7 @@ 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.joda.time.Duration; import org.junit.Rule; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java index 00c436d5cb3f..f8d01e9a016f 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java @@ -26,13 +26,13 @@ import java.util.Map; import org.apache.beam.sdk.runners.PTransformOverrideFactory.ReplacementOutput; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TaggedPValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; 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 021f19c347c6..f605e015cc09 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 @@ -38,10 +38,10 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; 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/WindowingStrategiesTest.java index 78ac61c8694e..7296a7789943 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/WindowingStrategiesTest.java @@ -29,8 +29,8 @@ import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Test; 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 c090001ce193..f3dd9a3e20eb 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 @@ -29,10 +29,10 @@ import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; /** * Static utility methods that provide {@link DoFnRunner} implementations. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFn.java index 311ed1c70530..a1605534d6a2 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFn.java @@ -24,7 +24,7 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SystemDoFnInternal; -import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java index 34afa5d3d4c3..2342c529342d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetDoFn.java @@ -22,8 +22,8 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SystemDoFnInternal; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowingStrategy; /** * A general {@link GroupAlsoByWindowsDoFn}. This delegates all of the logic to the 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 c553dbffed27..4c1fe9516e4c 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 @@ -27,9 +27,9 @@ import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java index 694c5ebe056e..fca3c76423d2 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java @@ -32,9 +32,9 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; /** * An implementation of {@link GroupByKey} built on top of a lower-level {@link GroupByKeyOnly} diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index 872ee3e06657..66385c116ffb 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -27,8 +27,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java index 982d693243a3..c45387bacd86 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java index 8df7ad005aa4..b6e8567c4b8a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java @@ -22,8 +22,8 @@ import org.apache.beam.sdk.state.ReadableState; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; /** * Tracks which windows have non-empty panes. Specifically, which windows have new elements since diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java index 8e2ff7368b1d..b1701ebf3f44 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java @@ -22,7 +22,7 @@ import org.apache.beam.sdk.state.Timers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index 550b9b9769b4..63e8e6d34f48 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -36,8 +36,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index a949eddfb116..a04006bcedb1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -57,10 +57,10 @@ import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.joda.time.Duration; import org.joda.time.Instant; 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 2567bbaa3b57..4057790788c1 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 @@ -56,9 +56,9 @@ import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java index 7454dc9248d4..cf9dff233565 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java @@ -39,9 +39,9 @@ import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; 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/SplittableParDo.java index 74ffeb635339..a4d96392bf2a 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/SplittableParDo.java @@ -48,13 +48,13 @@ 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.util.WindowingStrategy; 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; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java index 346694566084..28a9deef8795 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java index ce15244d795c..64f5d9b165cc 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFnTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFnTest.java index cb8d4943ccf3..a265eade4ffd 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFnTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowViaOutputBufferDoFnTest.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java index 054a2e255c2d..761ffb87024e 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java @@ -47,11 +47,11 @@ import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java index bf784272cee4..88c6ab6e7e1b 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index 2b400a40e9a5..381a493d9595 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -72,10 +72,10 @@ import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; 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 eea30e912e02..6fb0dcb03992 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 @@ -67,11 +67,11 @@ import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java index 335aedecd190..f9e0aaf40845 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java @@ -29,9 +29,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.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; 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 d793584ca446..fae911782d9d 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 @@ -45,8 +45,8 @@ import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; 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/SplittableParDoTest.java index 0bb25fcfa747..b93ff3a95375 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/SplittableParDoTest.java @@ -58,13 +58,13 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.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; import org.joda.time.Instant; import org.junit.Rule; 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 d7d8261cca5f..2ca8b058761b 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 @@ -40,9 +40,9 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java index 51f467ae8171..9a10f53c1bbe 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java @@ -52,8 +52,8 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.joda.time.Duration; import org.joda.time.Instant; 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 36050023b399..791615a2a194 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 @@ -29,9 +29,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; class DirectGroupByKey extends ForwardingPTransform>, PCollection>>> { 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 3cdf351d48f7..75ff1c7e9060 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 @@ -45,10 +45,10 @@ import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** 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 d00e4080f87e..83548589c1eb 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 @@ -48,10 +48,10 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** 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 a3a345f68008..e5a668043203 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 @@ -36,10 +36,10 @@ import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; class ParDoEvaluator implements TransformEvaluator { 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 b10d66922db8..c72bf61e2c5f 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 @@ -44,13 +44,13 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; 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; /** * A {@link PTransformOverrideFactory} that provides overrides for applications of a {@link ParDo} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java index cd459e4c0f5d..1e773c936617 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java @@ -41,8 +41,8 @@ import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowingStrategy; /** * An in-process container for {@link PCollectionView PCollectionViews}, which provides methods for 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 fb3a96279b03..2c22caf5606b 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 @@ -41,11 +41,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; 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 ca302b8b11a6..02472b3e69f6 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 @@ -48,12 +48,12 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; /** A {@link TransformEvaluatorFactory} for stateful {@link ParDo}. */ final class StatefulParDoEvaluatorFactory implements TransformEvaluatorFactory { 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 b5486c0b0b0d..ea989d754668 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 @@ -40,13 +40,13 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java index fcefc5ff1df3..ece5a56732fd 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java @@ -26,7 +26,7 @@ import java.util.concurrent.Executor; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java index 077cd43213c3..3a7fdf3a913b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java @@ -32,10 +32,10 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; 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 63864f0856e2..123e9f233cfa 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 @@ -56,11 +56,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.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.WindowingStrategy; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java index 74e70f880e87..641575742cb1 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java @@ -41,11 +41,11 @@ 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.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; 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; import org.junit.Before; import org.junit.Rule; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java index f4de8839c28c..d7c444048165 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java @@ -47,11 +47,11 @@ import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; 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 95bbc36ac145..f6b652d3836f 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 @@ -60,13 +60,13 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.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; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java index 6875e1a939cb..a36787af86fc 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java @@ -40,9 +40,9 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View.CreatePCollectionView; import org.apache.beam.sdk.util.PCollectionViews; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java index 8d6e73f54f41..f4a53da28368 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java @@ -32,8 +32,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java index f28c8cf3aede..6fffd1a479ec 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java @@ -55,12 +55,12 @@ import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.PCollectionViews; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index 6a7689aa3855..205cd3ed9107 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -64,12 +64,12 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RichGroupReduceFunction; 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 bb86cd9dc98f..e96d1c5ec5e4 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 @@ -26,11 +26,11 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; 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.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; 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 7339c01d9b81..3cf76832447d 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 @@ -68,12 +68,12 @@ import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java index 6e27057f7ec3..4872a0646061 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java @@ -26,8 +26,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.util.Collector; /** 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 28e1a44d6a3d..9205bce1a83a 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 @@ -30,9 +30,9 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java index 9ccf0793ffa6..13be91312497 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java @@ -24,9 +24,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index 4099f52693ba..db12a49e76f2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -24,9 +24,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichGroupCombineFunction; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java index 90dcbff0e8f2..53d71d85c376 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java @@ -24,9 +24,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java index c317182ffde9..fa95477e1013 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java @@ -26,9 +26,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RuntimeContext; /** 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 9f000e0f23df..6517bf2c1a9f 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 @@ -40,10 +40,10 @@ 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.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java index 7ee2f6977de6..942bf42f560c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java @@ -34,8 +34,8 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; import org.joda.time.Instant; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java index 4aacb4a32e28..fb4c67857673 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java @@ -31,8 +31,8 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.util.Collector; import org.joda.time.Instant; 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 5287b85d0bd6..1844d6d9ce9c 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 @@ -67,9 +67,9 @@ import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider; 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 1887a992d64f..7d54cfa51d5d 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 @@ -41,9 +41,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.streaming.api.operators.InternalTimer; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index 3899303efd9b..bf64edefbaea 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -36,10 +36,10 @@ 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.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.streaming.api.operators.InternalTimer; /** diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java index 7519dbf9d5c1..8382a2d34a3b 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java @@ -53,9 +53,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.commons.lang3.SerializationUtils; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeHint; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java index 80dfa07288c4..79bc0e0b1671 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java @@ -54,10 +54,10 @@ 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.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java index ffd56c99ecfb..572b00561770 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java @@ -24,8 +24,8 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; /** * A primitive {@link PTransform} that implements the {@link Window#into(WindowFn)} diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java index 119c9c9a6440..13ae9ee03b8d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java @@ -38,13 +38,13 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index b85b315da869..1ff8a3f3affd 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -78,7 +78,6 @@ import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; @@ -87,6 +86,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.WindowingStrategy; /** * Dataflow batch overrides for {@link CreatePCollectionView}, specialized for different view types. 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 e727433eac08..92f9c9823410 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 @@ -91,7 +91,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -100,6 +99,7 @@ import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 1a806b9faecb..d5e650ef2a79 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 @@ -123,7 +123,6 @@ import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.ValueWithRecordId; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -133,6 +132,7 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java index aa9d9f8c2fee..d33fdfee38ef 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java @@ -31,9 +31,9 @@ import org.apache.beam.sdk.util.IdentityWindowFn; import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.ReshuffleTrigger; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; /** 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 55c62aea79b7..bd2742f147da 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 @@ -21,9 +21,9 @@ import java.util.Map; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; /** * Wrapper class holding the necessary information to serialize a {@link DoFn}. diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index 986ed5c757a1..48cb79f1731d 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -98,7 +98,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.GcsPathValidator; import org.apache.beam.sdk.util.GcsUtil; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -106,6 +105,7 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowingStrategy; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 5aebf29d1033..3d477261fd7a 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java @@ -86,10 +86,10 @@ import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.NoopPathValidator; import org.apache.beam.sdk.util.ReleaseInfo; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Description; import org.hamcrest.Matchers; import org.hamcrest.TypeSafeMatcher; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java index c9c7806084b9..fc2ff3d284e7 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java @@ -33,11 +33,11 @@ import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.NoopPathValidator; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.junit.Before; import org.junit.Rule; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java index 455868367ff7..f0dbd6183e60 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java @@ -29,12 +29,12 @@ import org.apache.beam.sdk.transforms.windowing.InvalidWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.NoopPathValidator; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Before; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java index f2e0bb350639..fdcea99b9165 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java @@ -31,10 +31,10 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; 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 4a2851d87cb5..9bc876018d45 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 @@ -50,9 +50,9 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java index 6a67ccee6e9d..d19094d1da87 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java @@ -26,8 +26,8 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.HashPartitioner; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaPairRDD; 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 410b7deb6e40..9bfd2fa3d573 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 @@ -33,9 +33,9 @@ import org.apache.beam.runners.spark.util.SparkSideInputReader; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.Accumulator; import org.apache.spark.api.java.function.PairFlatMapFunction; import scala.Tuple2; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java index 7d06d6bf849d..df633b098e4f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java @@ -37,10 +37,10 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGlobalCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGlobalCombineFn.java index 7d026c68b2c3..d0e90387eb40 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGlobalCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGlobalCombineFn.java @@ -33,9 +33,9 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; 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 9ee52de6262f..a70885bec06f 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 @@ -37,9 +37,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.Accumulator; import org.apache.spark.api.java.function.FlatMapFunction; import org.joda.time.Instant; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkKeyedCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkKeyedCombineFn.java index 58db8e47d13f..7ac8e7d4904e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkKeyedCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkKeyedCombineFn.java @@ -33,9 +33,9 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java index 30b51e66e4df..0ecfa7571ca4 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/StorageLevelPTransform.java @@ -20,8 +20,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; /** * Get RDD storage level for the input PCollection (mostly used for testing purpose). 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 d249e78df841..77d2c0e693e6 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 @@ -57,12 +57,12 @@ import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java index 7b6f9edeecf8..f462e60ac478 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java @@ -38,10 +38,10 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function; 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 9af4af2a87c7..08f0e172b638 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 @@ -74,12 +74,12 @@ import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java index 7580da77a138..299f5ba97a88 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java @@ -19,10 +19,10 @@ import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.WindowingStrategy; /** * A {@link PTransform} wrapping another transform. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java index d6e1a94a592f..e876e12ec717 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java @@ -27,10 +27,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java index 0e269a2955bc..dbd9b86758d6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java @@ -23,10 +23,10 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.NameUtils; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java index 9efac69671c7..df4242872905 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java @@ -28,7 +28,7 @@ public class TimerSpecs { public static TimerSpec timer(TimeDomain timeDomain) { - return new org.apache.beam.sdk.util.AutoValue_TimerSpecs_SimpleTimerSpec(timeDomain); + return new AutoValue_TimerSpecs_SimpleTimerSpec(timeDomain); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java index 85b8c5f3a937..b5d7db5fc204 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java @@ -65,7 +65,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -73,6 +72,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java index a648767f7202..446c8a5d1974 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java @@ -29,11 +29,11 @@ import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index 9d0e97ab16aa..1be948f779fb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -59,7 +59,6 @@ import org.apache.beam.sdk.util.NameUtils.NameOverride; import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; @@ -69,6 +68,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowingStrategy; /** * {@code PTransform}s for combining {@code PCollection} elements diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java index 7b282b535469..25d9c0519727 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java @@ -21,10 +21,10 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableLikeCoder; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.WindowingStrategy; /** * {@code Flatten} takes multiple {@code PCollection}s bundled diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java index 6a89c5ff3566..7516b255bb56 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java @@ -28,10 +28,10 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.WindowingStrategy; /** * {@code GroupByKey} takes a {@code PCollection>}, 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 d010d1f94ab9..27771912f2dd 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 @@ -30,10 +30,10 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.joda.time.Duration; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java index b16aadcf5bdb..71ebba5d8e0f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowingStrategy; /** * A {@link GlobalCombineFn} with a fixed accumulator coder. This is created from a 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 d5f21814bbaa..a61e3a6482dc 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 @@ -29,6 +29,7 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java index f2052accf0f2..a07bc5e5818b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java @@ -46,6 +46,7 @@ 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; /** * Implementations of {@link PCollectionView} shared across the SDK. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java index 706e039582b6..887f01103289 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java @@ -27,6 +27,7 @@ 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.WindowingStrategy; import org.joda.time.Duration; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java index 16223223749c..1095fb8c7558 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java @@ -37,7 +37,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowingStrategy; /** * A {@link PCollection PCollection<T>} is an immutable collection of values of type diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java index 5027df652ddf..d1bb6d7b5667 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionTuple.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection.IsBounded; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java index 71efc09ccdfc..f89041a6124e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java @@ -28,7 +28,6 @@ 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.util.WindowingStrategy; /** * A {@link PCollectionView PCollectionView<T>} is an immutable view of a {@link PCollection} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowingStrategy.java similarity index 99% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowingStrategy.java index 14f818afd581..8a773e23f9b5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowingStrategy.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.values; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; 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 e495758c788e..125e15902d17 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 @@ -44,7 +44,6 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.ParDo.MultiOutput; import org.apache.beam.sdk.transforms.ParDo.SingleOutput; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; @@ -57,6 +56,7 @@ import org.apache.beam.sdk.values.TaggedPValue; 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.Before; import org.junit.Rule; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java index 6c3aba272162..e7b680ab6c0a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java @@ -40,11 +40,11 @@ import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Sample; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.WindowingStrategy; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java index 5f71cabb6148..adf27f8f5279 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java @@ -35,11 +35,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValueBase; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index a980f87b5c74..f74d67344473 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -35,11 +35,11 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.PCollectionViews; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; 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 0556199eecfc..50e9c1d4732e 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 @@ -55,12 +55,12 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.Reshuffle; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java index b3fa2c616979..e72c5401ea81 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java @@ -55,13 +55,13 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.InvalidWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; 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 534e2309c468..92f6a9c01457 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 @@ -52,11 +52,11 @@ import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; 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.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java index eae465c73753..71bfdb5c7585 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java @@ -43,6 +43,7 @@ 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.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java index b704d07f0ad2..58e2bbdc859a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/PCollectionTupleTest.java @@ -40,7 +40,6 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.hamcrest.Matchers; import org.junit.Rule; diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoder.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoder.java index 30570508cfc3..d4e6f63f116c 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoder.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoder.java @@ -23,7 +23,6 @@ import java.io.InputStream; import java.io.OutputStream; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.util.VarInt; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index f29aeb9f3ce7..46bd6d437e42 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -61,8 +61,8 @@ import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 6d5e230b9c20..3e06369320af 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 @@ -75,9 +75,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; import org.junit.Before; import org.junit.Rule; import org.junit.Test; 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 b893ad5a5189..38da2d94c112 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 @@ -123,13 +123,13 @@ import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.CoreMatchers; import org.hamcrest.Matchers; import org.joda.time.Instant; From 1d4b1ed49ce2212d961ad37325a3bfce8964f673 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 3 May 2017 19:36:45 -0700 Subject: [PATCH 055/387] Move PathValidator adjacent to other filesystem bits --- .../beam/runners/dataflow/DataflowRunner.java | 2 +- .../beam/sdk/io/fs/NoopPathValidator.java | 52 +++++++++++++++++++ .../sdk/{util => io/fs}/PathValidator.java | 9 ++-- .../beam/sdk/util/NoopPathValidator.java | 4 +- .../extensions/gcp/options/GcpOptions.java | 2 +- .../extensions/gcp/options/GcsOptions.java | 2 +- .../beam/sdk/util/GcsPathValidator.java | 1 + 7 files changed, 65 insertions(+), 7 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/NoopPathValidator.java rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => io/fs}/PathValidator.java (87%) 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 d5e650ef2a79..ce824c66843b 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 @@ -88,6 +88,7 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.WriteFiles; +import org.apache.beam.sdk.io.fs.PathValidator; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; @@ -118,7 +119,6 @@ import org.apache.beam.sdk.util.InstanceBuilder; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.NameUtils; -import org.apache.beam.sdk.util.PathValidator; import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.ValueWithRecordId; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/NoopPathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/NoopPathValidator.java new file mode 100644 index 000000000000..d5be8f0fc5fb --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/NoopPathValidator.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.fs; + +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.options.PipelineOptions; + +/** + * For internal use only; no backwards compatibility guarantees. + * + *

    Noop implementation of {@link PathValidator}. All paths are allowed and returned unchanged. + */ +@Internal +public class NoopPathValidator implements PathValidator { + + private NoopPathValidator() { + } + + public static PathValidator fromOptions( + @SuppressWarnings("unused") PipelineOptions options) { + return new NoopPathValidator(); + } + + @Override + public void validateInputFilePatternSupported(String filepattern) {} + + @Override + public void validateOutputFilePrefixSupported(String filePrefix) {} + + @Override + public void validateOutputResourceSupported(ResourceId resourceId) {} + + @Override + public String verifyPath(String path) { + return path; + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/PathValidator.java similarity index 87% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/PathValidator.java index e69648b0d2da..b88a33e75727 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PathValidator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/PathValidator.java @@ -15,13 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.io.fs; -import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.annotations.Internal; /** - * Interface for controlling validation of paths. + * For internal use only; no backwards compatibility guarantees. + * + *

    Interface for controlling validation of paths. */ +@Internal public interface PathValidator { /** * Validate that a file pattern is conforming. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java index 1f3f5a848dd2..0015e5945767 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopPathValidator.java @@ -17,12 +17,14 @@ */ package org.apache.beam.sdk.util; +import org.apache.beam.sdk.io.fs.PathValidator; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; /** - * Noop implementation of {@link PathValidator}. All paths are allowed and returned unchanged. + * @deprecated use {@link org.apache.beam.sdk.io.fs.NoopPathValidator}. */ +@Deprecated public class NoopPathValidator implements PathValidator { private NoopPathValidator() { diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java index 445f00fd5c5a..126b7952d8d2 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java @@ -48,13 +48,13 @@ import org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory; import org.apache.beam.sdk.extensions.gcp.auth.GcpCredentialFactory; import org.apache.beam.sdk.extensions.gcp.auth.NullCredentialInitializer; +import org.apache.beam.sdk.io.fs.PathValidator; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.PathValidator; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; import org.apache.beam.sdk.util.Transport; import org.apache.beam.sdk.util.gcsfs.GcsPath; diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java index 954092c145c6..78e233e967c1 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcsOptions.java @@ -26,6 +26,7 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; +import org.apache.beam.sdk.io.fs.PathValidator; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; @@ -35,7 +36,6 @@ import org.apache.beam.sdk.util.GcsPathValidator; import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.PathValidator; /** * Options used to configure Google Cloud Storage. diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java index 9ad4152ca090..c4e557bd41de 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/GcsPathValidator.java @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; +import org.apache.beam.sdk.io.fs.PathValidator; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.gcsfs.GcsPath; From 6542eafcf3a84105d2716cd0200d40ccdf764472 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 3 May 2017 19:46:59 -0700 Subject: [PATCH 056/387] Move SideInputReader to runners/core-java --- .../runners/apex/ApexRunnerRegistrar.java | 1 - .../beam/runners/apex/ApexRunnerResult.java | 2 - .../beam/runners/apex/ApexYarnLauncher.java | 2 - .../translation/ReadUnboundedTranslator.java | 1 - .../apex/translation/TransformTranslator.java | 1 - .../operators/ApexGroupByKeyOperator.java | 2 +- .../operators/ApexParDoOperator.java | 4 +- .../ApexReadUnboundedInputOperator.java | 2 - .../utils/CoderAdapterStreamCodec.java | 2 - .../utils/ValueAndCoderKryoSerializable.java | 2 - .../apex/translation/utils/ValuesSource.java | 2 - .../runners/apex/ApexYarnLauncherTest.java | 2 - .../apex/examples/UnboundedTextSource.java | 2 - .../runners/apex/examples/WordCountTest.java | 2 - .../ApexGroupByKeyOperatorTest.java | 2 - .../translation/utils/CollectionSource.java | 2 - .../apache/beam/runners/core/DoFnRunners.java | 2 - .../runners/core/GlobalCombineFnRunner.java | 1 - .../runners/core/GlobalCombineFnRunners.java | 41 +++++++++++++++---- .../GroupAlsoByWindowViaWindowSetNewDoFn.java | 1 - .../runners/core}/NullSideInputReader.java | 2 +- ...oundedSplittableProcessElementInvoker.java | 1 - .../beam/runners/core/ProcessFnRunner.java | 1 - .../core}/ReadyCheckingSideInputReader.java | 2 +- .../runners/core/ReduceFnContextFactory.java | 1 - .../beam/runners/core/ReduceFnRunner.java | 1 - .../beam/runners/core/SideInputHandler.java | 1 - .../beam/runners/core}/SideInputReader.java | 2 +- .../beam/runners/core/SimpleDoFnRunner.java | 1 - .../runners/core/SimpleOldDoFnRunner.java | 1 - .../SimplePushbackSideInputDoFnRunner.java | 1 - .../core/UnsupportedSideInputReader.java | 1 - .../core/WindowingInternalsAdapters.java | 1 - ...edSplittableProcessElementInvokerTest.java | 1 - .../beam/runners/core/ReduceFnRunnerTest.java | 1 - .../beam/runners/core/ReduceFnTester.java | 2 - .../runners/core/SimpleDoFnRunnerTest.java | 1 - ...SimplePushbackSideInputDoFnRunnerTest.java | 1 - .../runners/core/SplittableParDoTest.java | 1 - .../runners/core/StatefulDoFnRunnerTest.java | 1 - .../runners/direct/EvaluationContext.java | 4 +- .../beam/runners/direct/ParDoEvaluator.java | 2 +- .../runners/direct/SideInputContainer.java | 4 +- ...ttableProcessElementsEvaluatorFactory.java | 2 +- .../runners/direct/EvaluationContextTest.java | 2 +- .../runners/direct/ParDoEvaluatorTest.java | 2 +- .../direct/SideInputContainerTest.java | 4 +- .../StatefulParDoEvaluatorFactoryTest.java | 2 +- .../functions/AbstractFlinkCombineRunner.java | 2 +- .../functions/FlinkSideInputReader.java | 2 +- .../functions/HashingFlinkCombineRunner.java | 2 +- .../functions/SortingFlinkCombineRunner.java | 2 +- .../wrappers/streaming/DoFnOperator.java | 4 +- .../runners/spark/SparkRunnerDebugger.java | 1 - .../spark/aggregators/NamedAggregators.java | 2 - .../runners/spark/coders/CoderHelpers.java | 1 - .../spark/coders/StatelessJavaSerializer.java | 1 - .../spark/io/SparkUnboundedSource.java | 1 - .../spark/metrics/AggregatorMetric.java | 1 - .../spark/metrics/SparkBeamMetricSource.java | 1 - .../spark/stateful/StateSpecFunctions.java | 1 - .../translation/SparkAbstractCombineFn.java | 2 +- .../spark/translation/TranslationUtils.java | 1 - .../spark/util/GlobalWatermarkHolder.java | 1 - .../spark/util/SparkSideInputReader.java | 2 +- .../beam/sdk/util/CombineContextFactory.java | 25 ----------- .../harness/control/ProcessBundleHandler.java | 2 +- 67 files changed, 60 insertions(+), 116 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/NullSideInputReader.java (98%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/ReadyCheckingSideInputReader.java (97%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/core-java/src/main/java/org/apache/beam/runners/core}/SideInputReader.java (97%) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java index aa6ef4546b23..8cde692f8c15 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java @@ -20,7 +20,6 @@ import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableList; - import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunner; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java index 41fdb7577dad..cc24ddd82a94 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java @@ -18,9 +18,7 @@ package org.apache.beam.runners.apex; import com.datatorrent.api.DAG; - import java.io.IOException; - import org.apache.apex.api.Launcher.AppHandle; import org.apache.apex.api.Launcher.ShutdownMode; import org.apache.beam.sdk.Pipeline; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexYarnLauncher.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexYarnLauncher.java index b84144cd251c..18d8e940ee60 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexYarnLauncher.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexYarnLauncher.java @@ -25,7 +25,6 @@ import com.datatorrent.api.StreamingApplication; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; - import java.io.BufferedReader; import java.io.ByteArrayOutputStream; import java.io.File; @@ -56,7 +55,6 @@ import java.util.Set; import java.util.jar.JarFile; import java.util.jar.Manifest; - import org.apache.apex.api.EmbeddedAppLauncher; import org.apache.apex.api.Launcher; import org.apache.apex.api.Launcher.AppHandle; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ReadUnboundedTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ReadUnboundedTranslator.java index b3034ac472e2..168cbf5c221c 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ReadUnboundedTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ReadUnboundedTranslator.java @@ -19,7 +19,6 @@ package org.apache.beam.runners.apex.translation; import com.datatorrent.api.InputOperator; - import org.apache.beam.runners.apex.translation.operators.ApexReadUnboundedInputOperator; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.UnboundedSource; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TransformTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TransformTranslator.java index eb810521c98a..49ff49b4dbf3 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TransformTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TransformTranslator.java @@ -20,7 +20,6 @@ import java.io.Serializable; - import org.apache.beam.sdk.transforms.PTransform; /** 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 9d4e9a235a6b..85836ad2acbf 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 @@ -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.SerializablePipelineOptions; +import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.runners.core.OutputWindowedValue; import org.apache.beam.runners.core.ReduceFnRunner; import org.apache.beam.runners.core.StateInternalsFactory; @@ -49,7 +50,6 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; 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 99ad9647322a..8c516b1eb9ca 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 @@ -43,8 +43,10 @@ import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItemCoder; +import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; 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.StateInternals; import org.apache.beam.runners.core.StatefulDoFnRunner; @@ -56,8 +58,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.util.NullSideInputReader; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexReadUnboundedInputOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexReadUnboundedInputOperator.java index ac28c2a8e338..1549560f56fe 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexReadUnboundedInputOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexReadUnboundedInputOperator.java @@ -26,9 +26,7 @@ import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; import com.esotericsoftware.kryo.serializers.JavaSerializer; import com.google.common.base.Throwables; - import java.io.IOException; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple.DataTuple; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/CoderAdapterStreamCodec.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/CoderAdapterStreamCodec.java index d08e76f09657..f6ce1d075b22 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/CoderAdapterStreamCodec.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/CoderAdapterStreamCodec.java @@ -19,12 +19,10 @@ import com.datatorrent.api.StreamCodec; import com.datatorrent.netlet.util.Slice; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.Serializable; - import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValueAndCoderKryoSerializable.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValueAndCoderKryoSerializable.java index 395ad1f3945d..2a72f1719346 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValueAndCoderKryoSerializable.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValueAndCoderKryoSerializable.java @@ -22,9 +22,7 @@ import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; import com.esotericsoftware.kryo.serializers.JavaSerializer; - import java.io.IOException; - import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValuesSource.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValuesSource.java index 62c92a004892..41f027faa96d 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValuesSource.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValuesSource.java @@ -24,9 +24,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.NoSuchElementException; - import javax.annotation.Nullable; - import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.IterableCoder; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexYarnLauncherTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexYarnLauncherTest.java index 6ffb0919dc42..68ec2ea79dcf 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexYarnLauncherTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexYarnLauncherTest.java @@ -26,7 +26,6 @@ import com.datatorrent.api.Context.DAGContext; import com.datatorrent.api.DAG; import com.datatorrent.api.StreamingApplication; - import java.io.File; import java.net.URI; import java.nio.file.FileSystem; @@ -37,7 +36,6 @@ import java.util.Map; import java.util.Properties; import java.util.jar.JarFile; - import org.apache.apex.api.EmbeddedAppLauncher; import org.apache.apex.api.Launcher; import org.apache.apex.api.Launcher.AppHandle; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/UnboundedTextSource.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/UnboundedTextSource.java index abe97f6de049..c590a2e9783c 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/UnboundedTextSource.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/UnboundedTextSource.java @@ -23,9 +23,7 @@ import java.util.Collections; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; - import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.UnboundedSource; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/WordCountTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/WordCountTest.java index 83af61b028a6..e76096ef78d1 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/WordCountTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/WordCountTest.java @@ -18,11 +18,9 @@ package org.apache.beam.runners.apex.examples; import com.google.common.collect.Sets; - import java.io.File; import java.util.HashSet; import java.util.concurrent.ConcurrentHashMap; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.ApexRunnerResult; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java index e31d83051dc5..206b43001734 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java @@ -20,9 +20,7 @@ import com.datatorrent.api.Sink; import com.datatorrent.lib.util.KryoCloneUtils; import com.google.common.collect.Lists; - import java.util.List; - import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.TestApexRunner; import org.apache.beam.runners.apex.translation.operators.ApexGroupByKeyOperator; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/CollectionSource.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/CollectionSource.java index 92812b4f4f5d..288aadedb46c 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/CollectionSource.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/CollectionSource.java @@ -25,9 +25,7 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; - import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; 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 f3dd9a3e20eb..ee3aefa1054b 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 @@ -26,8 +26,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunner.java index 5325ba63d1e1..4c312b43491b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunner.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.SideInputReader; /** * An interface that runs a {@link GlobalCombineFn} with unified APIs. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunners.java index d45b5031ee91..d98bac83d496 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunners.java @@ -22,10 +22,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn; +import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.CombineContextFactory; -import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.values.PCollectionView; /** * Static utility methods that provide {@link GlobalCombineFnRunner} implementations for different @@ -46,6 +46,33 @@ public static GlobalCombineFnRunner T sideInput(PCollectionView view) { + if (!sideInputReader.contains(view)) { + throw new IllegalArgumentException("calling sideInput() with unknown view"); + } + + BoundedWindow sideInputWindow = + view.getWindowMappingFn().getSideInputWindow(mainInputWindow); + return sideInputReader.get(view, sideInputWindow); + } + }; + } + /** * An implementation of {@link GlobalCombineFnRunner} with {@link CombineFn}. * @@ -136,7 +163,7 @@ public AccumT createAccumulator( SideInputReader sideInputReader, Collection windows) { return combineFnWithContext.createAccumulator( - CombineContextFactory.createFromComponents( + createFromComponents( options, sideInputReader, Iterables.getOnlyElement(windows))); } @@ -150,7 +177,7 @@ public AccumT addInput( return combineFnWithContext.addInput( accumulator, input, - CombineContextFactory.createFromComponents( + createFromComponents( options, sideInputReader, Iterables.getOnlyElement(windows))); } @@ -162,7 +189,7 @@ public AccumT mergeAccumulators( Collection windows) { return combineFnWithContext.mergeAccumulators( accumulators, - CombineContextFactory.createFromComponents( + createFromComponents( options, sideInputReader, Iterables.getOnlyElement(windows))); } @@ -174,7 +201,7 @@ public OutputT extractOutput( Collection windows) { return combineFnWithContext.extractOutput( accumulator, - CombineContextFactory.createFromComponents( + createFromComponents( options, sideInputReader, Iterables.getOnlyElement(windows))); } @@ -186,7 +213,7 @@ public AccumT compact( Collection windows) { return combineFnWithContext.compact( accumulator, - CombineContextFactory.createFromComponents( + createFromComponents( options, sideInputReader, Iterables.getOnlyElement(windows))); } } 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 4c1fe9516e4c..5b82d1fdfda4 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 @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/NullSideInputReader.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/NullSideInputReader.java index 3da2dc17b401..786ed41a80ef 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/NullSideInputReader.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import com.google.common.collect.Sets; import java.util.Collections; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index 16bdfa361c0d..2db6531e050c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -36,7 +36,6 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; 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 7cbf0d214053..61f413fd8cac 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 @@ -28,7 +28,6 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.joda.time.Instant; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReadyCheckingSideInputReader.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReadyCheckingSideInputReader.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReadyCheckingSideInputReader.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ReadyCheckingSideInputReader.java index cb38a55a0da4..8d1f0e277be1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReadyCheckingSideInputReader.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReadyCheckingSideInputReader.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index 63e8e6d34f48..bd327f312378 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -35,7 +35,6 @@ import org.apache.beam.sdk.state.Timers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index a04006bcedb1..d2ed835ba463 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -54,7 +54,6 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java index af750104e6dc..539b9f08c67c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java @@ -32,7 +32,6 @@ import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputReader.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputReader.java index e81c704cf196..7d1b829710f2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputReader.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.core; import javax.annotation.Nullable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; 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 4057790788c1..aab34a51f5eb 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 @@ -52,7 +52,6 @@ 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.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java index cf9dff233565..2a0b688142b6 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleOldDoFnRunner.java @@ -35,7 +35,6 @@ 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.SideInputReader; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java index 36a04d9e2f04..3f77f7d988f0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java @@ -25,7 +25,6 @@ import java.util.Set; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnsupportedSideInputReader.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnsupportedSideInputReader.java index 4230f8c75c4c..c1c717952b05 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnsupportedSideInputReader.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnsupportedSideInputReader.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.core; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowingInternalsAdapters.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowingInternalsAdapters.java index 1b36bf9c4872..4a5844562980 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowingInternalsAdapters.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowingInternalsAdapters.java @@ -20,7 +20,6 @@ import java.util.Collection; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Instant; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java index 541e2383913e..a2f6acc9cdeb 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java @@ -33,7 +33,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Duration; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index 381a493d9595..3dd98e0b7cf7 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -70,7 +70,6 @@ import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; 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 6fb0dcb03992..7de8f3b28dfd 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 @@ -62,9 +62,7 @@ import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; 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 fae911782d9d..abefd1c488b7 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 @@ -42,7 +42,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java index dabc4f013b86..79bf0b29f797 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java @@ -39,7 +39,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IdentitySideInputWindowFn; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; 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/SplittableParDoTest.java index b93ff3a95375..be4cf08b662d 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/SplittableParDoTest.java @@ -56,7 +56,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; 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 2ca8b058761b..76351e40cad3 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 @@ -38,7 +38,6 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; 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 75ff1c7e9060..7b6461103477 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 @@ -32,6 +32,8 @@ 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; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; @@ -42,8 +44,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; 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 e5a668043203..4cfd16fec901 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 @@ -26,6 +26,7 @@ import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; +import org.apache.beam.runners.core.ReadyCheckingSideInputReader; import org.apache.beam.runners.core.SimplePushbackSideInputDoFnRunner; import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; @@ -33,7 +34,6 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java index 1e773c936617..380dc65b0090 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java @@ -35,11 +35,11 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; +import org.apache.beam.runners.core.ReadyCheckingSideInputReader; +import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.PCollectionViewWindow; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.WindowingStrategy; 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 2c22caf5606b..44f2e853f97c 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 @@ -27,6 +27,7 @@ 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.StateInternals; @@ -39,7 +40,6 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; 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 123e9f233cfa..077bb6865cfc 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 @@ -31,6 +31,7 @@ import java.util.Collections; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.core.StateTags; @@ -54,7 +55,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; 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 ef8add968180..88fd5d2f159d 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 @@ -30,6 +30,7 @@ import java.util.Collection; import java.util.List; import javax.annotation.Nullable; +import org.apache.beam.runners.core.ReadyCheckingSideInputReader; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.sdk.testing.TestPipeline; @@ -44,7 +45,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IdentitySideInputWindowFn; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java index d7c444048165..d4ca9fdd76b7 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java @@ -32,6 +32,8 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import org.apache.beam.runners.core.ReadyCheckingSideInputReader; +import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; @@ -44,8 +46,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.util.PCollectionViews; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; 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 f6b652d3836f..f1ba57cf8c49 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 @@ -35,6 +35,7 @@ import java.util.List; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.KeyedWorkItems; +import org.apache.beam.runners.core.ReadyCheckingSideInputReader; import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; @@ -58,7 +59,6 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java index 4872a0646061..2ae7833764e1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java @@ -21,10 +21,10 @@ import java.util.Collection; import org.apache.beam.runners.core.GlobalCombineFnRunner; import org.apache.beam.runners.core.GlobalCombineFnRunners; +import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java index fa95477e1013..f2752907b70a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java @@ -23,8 +23,8 @@ import java.util.HashMap; import java.util.Map; import javax.annotation.Nullable; +import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java index 942bf42f560c..feb8b3931865 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java @@ -27,12 +27,12 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java index fb4c67857673..026a35c85cb1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java @@ -23,13 +23,13 @@ import java.util.Comparator; import java.util.Iterator; import java.util.List; +import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; 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 1844d6d9ce9c..f35ba7a0b086 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 @@ -34,8 +34,10 @@ 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; 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.StateInternals; import org.apache.beam.runners.core.StateNamespace; @@ -64,8 +66,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.NullSideInputReader; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java index 8d47e1a31e82..11c52c7a2ff3 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java @@ -20,7 +20,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; - import org.apache.beam.runners.spark.translation.EvaluationContext; import org.apache.beam.runners.spark.translation.SparkPipelineTranslator; import org.apache.beam.runners.spark.translation.TransformTranslator; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java index cf6c9ade8e13..c836ca5801b8 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java @@ -22,14 +22,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; - import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Map; import java.util.TreeMap; - import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java index 9c46ecf379db..85e3572b1de4 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java @@ -32,7 +32,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.PairFunction; - import scala.Tuple2; /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/StatelessJavaSerializer.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/StatelessJavaSerializer.java index 01b3b93c6869..0cf4951dc4f7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/StatelessJavaSerializer.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/StatelessJavaSerializer.java @@ -23,7 +23,6 @@ import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; - import java.io.IOException; import java.io.InputStream; import java.io.ObjectInputStream; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java index 6b34590be1e6..0388f6c1f1e8 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java @@ -58,7 +58,6 @@ import org.apache.spark.streaming.dstream.DStream; import org.apache.spark.streaming.scheduler.StreamInputInfo; import org.joda.time.Instant; - import scala.Tuple2; import scala.runtime.BoxedUnit; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java index 271cc6bff867..450bc95939fd 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java @@ -19,7 +19,6 @@ package org.apache.beam.runners.spark.metrics; import com.codahale.metrics.Metric; - import org.apache.beam.runners.spark.aggregators.NamedAggregators; /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetricSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetricSource.java index 9cab66dae2f6..5c6fc241265c 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetricSource.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetricSource.java @@ -19,7 +19,6 @@ package org.apache.beam.runners.spark.metrics; import com.codahale.metrics.MetricRegistry; - import org.apache.spark.metrics.source.Source; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java index 30ee639b2b67..d8d52c425ecb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java @@ -47,7 +47,6 @@ import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import scala.Option; import scala.Tuple2; import scala.runtime.AbstractFunction3; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java index df633b098e4f..315f7fb18ddd 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java @@ -29,13 +29,13 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.runners.spark.util.SparkSideInputReader; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java index f462e60ac478..993062c048d4 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java @@ -50,7 +50,6 @@ import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; - import scala.Tuple2; /** A set of utilities to help translating Beam transformations into Spark transformations. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java index 212f974211b8..8b384d8e2ab0 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java @@ -26,7 +26,6 @@ import java.util.Map; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; - import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java index e876e12ec717..6c91088c2c3f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java @@ -24,8 +24,8 @@ import com.google.common.collect.Iterables; import java.util.Map; import javax.annotation.Nullable; +import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java index f93cb0b50200..b72fd8280ea4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java @@ -20,7 +20,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.state.StateContext; import org.apache.beam.sdk.transforms.CombineWithContext.Context; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollectionView; /** @@ -64,28 +63,4 @@ public T sideInput(PCollectionView view) { }; } - /** - * Returns a {@code Combine.Context} from {@code PipelineOptions}, {@code SideInputReader}, - * and the main input window. - */ - public static Context createFromComponents(final PipelineOptions options, - final SideInputReader sideInputReader, final BoundedWindow mainInputWindow) { - return new Context() { - @Override - public PipelineOptions getPipelineOptions() { - return options; - } - - @Override - public T sideInput(PCollectionView view) { - if (!sideInputReader.contains(view)) { - throw new IllegalArgumentException("calling sideInput() with unknown view"); - } - - BoundedWindow sideInputWindow = - view.getWindowMappingFn().getSideInputWindow(mainInputWindow); - return sideInputReader.get(view, sideInputWindow); - } - }; - } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 46bd6d437e42..fd9f0dfb534d 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -53,12 +53,12 @@ import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.core.DoFnRunners.OutputManager; +import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.runners.dataflow.util.DoFnInfo; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; From e0b3f8064b97d8678e75bf6ba25244bca31e6a7d Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 3 May 2017 19:57:02 -0700 Subject: [PATCH 057/387] Move Reshuffle and ReshuffleTrigger out of util These are two deprecated, but regularly used, transforms and triggers, to the point where the Runner API proto even includes something akin to the ReshuffleTrigger. They are moved to where they "belong" and marked deprecated. --- .../runners/core/construction/Triggers.java | 2 +- .../ReshuffleTriggerStateMachine.java | 2 +- .../flink/FlinkBatchTransformTranslators.java | 2 +- .../FlinkStreamingTransformTranslators.java | 2 +- .../beam/runners/dataflow/DataflowRunner.java | 2 +- .../dataflow/ReshuffleOverrideFactory.java | 4 +-- .../translation/GroupCombineFunctions.java | 3 ++- .../translation/TransformTranslator.java | 2 +- .../StreamingTransformTranslator.java | 2 +- .../org/apache/beam/sdk/io/FileSystems.java | 1 - .../{util => transforms}/ReifyTimestamps.java | 7 ++--- .../sdk/{util => transforms}/Reshuffle.java | 27 ++++++++++--------- .../windowing}/ReshuffleTrigger.java | 16 +++++++---- .../beam/sdk/transforms/GroupByKeyTest.java | 1 - .../ReifyTimestampsTest.java | 7 +---- .../{util => transforms}/ReshuffleTest.java | 7 +---- .../windowing}/ReshuffleTriggerTest.java | 4 +-- .../extensions/sorter/ExternalSorterTest.java | 1 - .../apache/beam/sdk/io/common/HashingFn.java | 2 -- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 2 +- .../sdk/io/gcp/bigquery/BigQueryHelpers.java | 2 -- .../io/gcp/bigquery/BigQueryQuerySource.java | 1 - .../sdk/io/gcp/bigquery/CalculateSchemas.java | 1 - .../sdk/io/gcp/bigquery/CreateTables.java | 1 - .../io/gcp/bigquery/StreamingWriteTables.java | 2 +- .../sdk/io/gcp/bigquery/TableDestination.java | 1 - .../sdk/io/gcp/bigquery/WritePartition.java | 1 - .../beam/sdk/io/gcp/bigquery/WriteRename.java | 1 - .../beam/sdk/io/gcp/bigquery/WriteTables.java | 1 - .../io/gcp/bigquery/FakeBigQueryServices.java | 2 -- .../sdk/io/gcp/bigquery/FakeJobService.java | 3 --- .../sdk/io/gcp/bigquery/TableContainer.java | 1 - .../ConfigurableEmployeeInputFormat.java | 1 - .../inputformat/EmployeeInputFormat.java | 1 - .../ReuseObjectsEmployeeInputFormat.java | 1 - .../inputformat/TestEmployeeDataSet.java | 2 -- .../inputformat/HIFIOWithElasticTest.java | 1 - .../HIFIOWithEmbeddedCassandraTest.java | 2 -- .../integration/tests/HIFIOCassandraIT.java | 2 -- .../integration/tests/HIFIOElasticIT.java | 1 - .../org/apache/beam/sdk/io/jdbc/JdbcIO.java | 2 -- .../org/apache/beam/sdk/io/jdbc/JdbcIOIT.java | 1 - .../org/apache/beam/sdk/io/jms/JmsRecord.java | 1 - .../sdk/io/kafka/KafkaCheckpointMark.java | 2 -- .../apache/beam/sdk/io/kafka/KafkaRecord.java | 1 - .../CoderBasedKafkaDeserializer.java | 1 - .../CoderBasedKafkaSerializer.java | 1 - .../serialization/InstantDeserializer.java | 1 - .../serialization/InstantSerializer.java | 1 - .../beam/sdk/io/mongodb/MongoDbGridFSIO.java | 3 --- .../sdk/io/mongodb/MongoDBGridFSIOTest.java | 3 --- .../beam/sdk/io/mongodb/MongoDbIOTest.java | 4 --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 3 --- 53 files changed, 44 insertions(+), 104 deletions(-) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => transforms}/ReifyTimestamps.java (92%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => transforms}/Reshuffle.java (79%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => transforms/windowing}/ReshuffleTrigger.java (75%) rename sdks/java/core/src/test/java/org/apache/beam/sdk/{util => transforms}/ReifyTimestampsTest.java (93%) rename sdks/java/core/src/test/java/org/apache/beam/sdk/{util => transforms}/ReshuffleTest.java (97%) rename sdks/java/core/src/test/java/org/apache/beam/sdk/{util => transforms/windowing}/ReshuffleTriggerTest.java (90%) 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/Triggers.java index 5e735715b3ee..df6c9ed1fa04 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/Triggers.java @@ -40,10 +40,10 @@ import org.apache.beam.sdk.transforms.windowing.Never.NeverTrigger; import org.apache.beam.sdk.transforms.windowing.OrFinallyTrigger; import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger; import org.apache.beam.sdk.transforms.windowing.TimestampTransform; import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; -import org.apache.beam.sdk.util.ReshuffleTrigger; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java index 8a2b736b629d..61ede342d921 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachine.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.core.triggers; -import org.apache.beam.sdk.util.Reshuffle; +import org.apache.beam.sdk.transforms.Reshuffle; /** * The trigger used with {@link Reshuffle} which triggers on every element diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index 205cd3ed9107..3689698c1c61 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -53,6 +53,7 @@ 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.Reshuffle; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.join.UnionCoder; @@ -62,7 +63,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; 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 3cf76832447d..615eaea29735 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 @@ -56,6 +56,7 @@ 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.Reshuffle; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.join.UnionCoder; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; @@ -66,7 +67,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; 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 ce824c66843b..7123316f436e 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 @@ -110,6 +110,7 @@ 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.Reshuffle; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.View; @@ -120,7 +121,6 @@ import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.NameUtils; import org.apache.beam.sdk.util.ReleaseInfo; -import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.ValueWithRecordId; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java index d33fdfee38ef..5814efd0ce47 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java @@ -26,11 +26,11 @@ 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.Reshuffle; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IdentityWindowFn; -import org.apache.beam.sdk.util.Reshuffle; -import org.apache.beam.sdk.util.ReshuffleTrigger; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java index d19094d1da87..1fa46b5b634e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.KV; @@ -223,7 +224,7 @@ public static Optional>> combine /** * An implementation of - * {@link org.apache.beam.sdk.util.Reshuffle} for the Spark runner. + * {@link Reshuffle} for the Spark runner. */ public static JavaRDD>> reshuffle( JavaRDD>> rdd, 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 77d2c0e693e6..8a8e24677759 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 @@ -50,12 +50,12 @@ 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.Reshuffle; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; 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 08f0e172b638..2c4a7476d0ea 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 @@ -66,13 +66,13 @@ 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.Reshuffle; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; 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 a3af8d9860e9..2e11177b3184 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 @@ -33,7 +33,6 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.common.collect.TreeMultimap; - import java.io.FileNotFoundException; import java.io.IOException; import java.nio.channels.ReadableByteChannel; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestamps.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ReifyTimestamps.java similarity index 92% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestamps.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ReifyTimestamps.java index 3b291af01f21..0b1ab2588672 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestamps.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ReifyTimestamps.java @@ -16,11 +16,8 @@ * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.transforms; -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.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; @@ -29,7 +26,7 @@ * {@link PTransform PTransforms} for reifying the timestamp of values and reemitting the original * value with the original timestamp. */ -public class ReifyTimestamps { +class ReifyTimestamps { private ReifyTimestamps() {} /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java similarity index 79% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java index 887f01103289..5394826e8551 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Reshuffle.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java @@ -15,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.transforms; -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.annotations.Internal; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.IdentityWindowFn; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; @@ -31,18 +30,22 @@ import org.joda.time.Duration; /** - * A {@link PTransform} that returns a {@link PCollection} equivalent to its input but operationally - * provides some of the side effects of a {@link GroupByKey}, in particular preventing fusion of - * the surrounding transforms, checkpointing and deduplication by id (see - * {@link ValueWithRecordId}). + * For internal use only; no backwards compatibility guarantees. * - *

    Performs a {@link GroupByKey} so that the data is key-partitioned. Configures the - * {@link WindowingStrategy} so that no data is dropped, but doesn't affect the need for - * the user to specify allowed lateness and accumulation mode before a user-inserted GroupByKey. + *

    A {@link PTransform} that returns a {@link PCollection} equivalent to its input but + * operationally provides some of the side effects of a {@link GroupByKey}, in particular preventing + * fusion of the surrounding transforms, checkpointing and deduplication by id. + * + *

    Performs a {@link GroupByKey} so that the data is key-partitioned. Configures the {@link + * WindowingStrategy} so that no data is dropped, but doesn't affect the need for the user to + * specify allowed lateness and accumulation mode before a user-inserted GroupByKey. * * @param The type of key being reshuffled on. * @param The type of value being reshuffled. + * @deprecated this transform's intended side effects are not portable; it will likely be removed */ +@Internal +@Deprecated public class Reshuffle extends PTransform>, PCollection>> { private Reshuffle() { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/ReshuffleTrigger.java similarity index 75% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/ReshuffleTrigger.java index 8dd648adcd50..ceb7011a5254 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/ReshuffleTrigger.java @@ -15,19 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.transforms.windowing; import java.util.List; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.Trigger; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.transforms.Reshuffle; import org.joda.time.Instant; /** - * The trigger used with {@link Reshuffle} which triggers on every element - * and never buffers state. + * For internal use only; no backwards compatibility guarantees. + * + *

    The trigger used with {@link Reshuffle} which triggers on every element and never buffers + * state. * * @param The kind of window that is being reshuffled. + * @deprecated The intended side effect of {@link Reshuffle} is not portable; it will likely be + * removed */ +@Internal +@Deprecated public class ReshuffleTrigger extends Trigger { public ReshuffleTrigger() { 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 50e9c1d4732e..9cb642a77ff4 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 @@ -54,7 +54,6 @@ import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReifyTimestampsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTimestampsTest.java similarity index 93% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReifyTimestampsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTimestampsTest.java index 2942efdf38c7..181433eaafc1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReifyTimestampsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTimestampsTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.transforms; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; @@ -25,11 +25,6 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.ValidatesRunner; -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.SerializableFunction; -import org.apache.beam.sdk.transforms.WithTimestamps; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java similarity index 97% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java index 71bfdb5c7585..1038fd63810b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.transforms; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; @@ -30,11 +30,6 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.ValidatesRunner; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.Values; -import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/ReshuffleTriggerTest.java similarity index 90% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/ReshuffleTriggerTest.java index 63c71ed0b72c..5985ecbc209e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/ReshuffleTriggerTest.java @@ -15,12 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.transforms.windowing; import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.Trigger; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java index 689dbff2fb30..e98bfa74a23c 100644 --- a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java +++ b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java @@ -26,7 +26,6 @@ import java.nio.file.Path; import java.nio.file.SimpleFileVisitor; import java.nio.file.attribute.BasicFileAttributes; - import org.apache.beam.sdk.extensions.sorter.SorterTestUtils.SorterGenerator; import org.junit.AfterClass; import org.junit.BeforeClass; diff --git a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/HashingFn.java b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/HashingFn.java index d534c8765c85..0a7251711c9c 100644 --- a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/HashingFn.java +++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/HashingFn.java @@ -17,14 +17,12 @@ import com.google.common.collect.Lists; import com.google.common.hash.HashCode; import com.google.common.hash.Hashing; - import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.util.List; - import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; 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 f4221354621b..ba64ab108656 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 @@ -40,12 +40,12 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java index 318ea8910b8d..09508e0a4628 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java @@ -27,7 +27,6 @@ import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; import com.google.common.hash.Hashing; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -35,7 +34,6 @@ import java.util.UUID; import java.util.regex.Matcher; import javax.annotation.Nullable; - import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.ResolveOptions; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySource.java index 710c934a70b6..aee88e520857 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySource.java @@ -33,7 +33,6 @@ import java.io.ObjectInputStream; import java.util.List; import java.util.concurrent.atomic.AtomicReference; - import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.Status; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java index db172dc0b12e..1ac216f87e0f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CalculateSchemas.java @@ -25,7 +25,6 @@ import java.util.Map; import org.apache.beam.sdk.transforms.Distinct; import org.apache.beam.sdk.transforms.DoFn; - import org.apache.beam.sdk.transforms.Keys; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java index 210a072509a0..3dc10b08a024 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java @@ -29,7 +29,6 @@ import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; - import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.transforms.DoFn; 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 20b47e10f9a0..886236b5f00d 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 @@ -22,10 +22,10 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.Reshuffle; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java index 962e2cde7f4d..7a82c54e277a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java @@ -19,7 +19,6 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableReference; - import java.io.Serializable; import java.util.Objects; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java index 4136fa06084a..66004b22b6a5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java @@ -23,7 +23,6 @@ import java.util.List; import java.util.Map; import java.util.UUID; - import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; import org.apache.beam.sdk.io.gcp.bigquery.WriteBundlesToFiles.Result; import org.apache.beam.sdk.transforms.DoFn; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java index f575a3da1b77..bf9d9f1f8000 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java @@ -28,7 +28,6 @@ import java.util.List; import java.util.Map; import javax.annotation.Nullable; - import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.Status; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index e7dba2a87a96..83ff16bb67d1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -31,7 +31,6 @@ import java.util.List; import java.util.Map; import javax.annotation.Nullable; - import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MoveOptions; import org.apache.beam.sdk.io.fs.ResourceId; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java index 367aeb78ae10..18ff68889cca 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeBigQueryServices.java @@ -24,13 +24,11 @@ import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.common.collect.Lists; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.List; import java.util.NoSuchElementException; - import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.ListCoder; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java index ee3af0ba9892..50be0bb8156d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java @@ -42,7 +42,6 @@ import com.google.common.collect.HashBasedTable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.FileReader; @@ -54,7 +53,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; - import org.apache.avro.Schema; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericDatumWriter; @@ -69,7 +67,6 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; import org.apache.beam.sdk.util.FluentBackoff; - import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.Transport; import org.joda.time.Duration; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java index d52723b79bc0..89150698114c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java @@ -19,7 +19,6 @@ import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableRow; - import java.util.ArrayList; import java.util.List; diff --git a/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ConfigurableEmployeeInputFormat.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ConfigurableEmployeeInputFormat.java index 40f949baf7cf..1d3190eb130d 100644 --- a/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ConfigurableEmployeeInputFormat.java +++ b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ConfigurableEmployeeInputFormat.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; diff --git a/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/EmployeeInputFormat.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/EmployeeInputFormat.java index 206f9ab49f51..a641707c2fcf 100644 --- a/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/EmployeeInputFormat.java +++ b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/EmployeeInputFormat.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.beam.sdk.values.KV; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; diff --git a/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ReuseObjectsEmployeeInputFormat.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ReuseObjectsEmployeeInputFormat.java index fbe74ec05ada..defef472d5a9 100644 --- a/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ReuseObjectsEmployeeInputFormat.java +++ b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/ReuseObjectsEmployeeInputFormat.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; - import org.apache.beam.sdk.values.KV; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; diff --git a/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/TestEmployeeDataSet.java b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/TestEmployeeDataSet.java index 4a8fe951c347..4dd7004f2dfc 100644 --- a/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/TestEmployeeDataSet.java +++ b/sdks/java/io/hadoop/input-format/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/TestEmployeeDataSet.java @@ -16,10 +16,8 @@ import com.google.common.base.Function; import com.google.common.collect.Lists; - import java.util.ArrayList; import java.util.List; - import org.apache.beam.sdk.values.KV; import org.apache.hadoop.io.Text; /** 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 51cbd5a2d124..99d371df4f55 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 @@ -24,7 +24,6 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; - import org.apache.beam.sdk.io.common.HashingFn; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithEmbeddedCassandraTest.java b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithEmbeddedCassandraTest.java index 50b2c4f09947..fd3590c7808d 100644 --- a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithEmbeddedCassandraTest.java +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/HIFIOWithEmbeddedCassandraTest.java @@ -23,9 +23,7 @@ import com.datastax.driver.core.SocketOptions; import com.datastax.driver.mapping.annotations.Column; import com.datastax.driver.mapping.annotations.Table; - import java.io.Serializable; - import org.apache.beam.sdk.io.common.HashingFn; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOCassandraIT.java b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOCassandraIT.java index ab8203be0900..5a3caed2e9f5 100644 --- a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOCassandraIT.java +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOCassandraIT.java @@ -18,9 +18,7 @@ package org.apache.beam.sdk.io.hadoop.inputformat.integration.tests; import com.datastax.driver.core.Row; - import java.io.Serializable; - import org.apache.beam.sdk.io.common.HashingFn; import org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO; import org.apache.beam.sdk.io.hadoop.inputformat.custom.options.HIFTestOptions; diff --git a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOElasticIT.java b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOElasticIT.java index 08c0668e243a..f7f496debadf 100644 --- a/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOElasticIT.java +++ b/sdks/java/io/hadoop/jdk1.8-tests/src/test/java/org/apache/beam/sdk/io/hadoop/inputformat/integration/tests/HIFIOElasticIT.java @@ -16,7 +16,6 @@ import java.io.IOException; import java.io.Serializable; - import org.apache.beam.sdk.io.common.HashingFn; import org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO; import org.apache.beam.sdk.io.hadoop.inputformat.custom.options.HIFTestOptions; diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java index 2eb53dd6361a..8092da6b971a 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java @@ -27,10 +27,8 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.Random; - import javax.annotation.Nullable; import javax.sql.DataSource; - import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java index 3eaa5b77f6be..e8ffad6c56bb 100644 --- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java +++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java @@ -24,7 +24,6 @@ import java.sql.Statement; import java.util.ArrayList; import java.util.List; - import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java index 15a1109df3c6..1c3b3b380d97 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java @@ -20,7 +20,6 @@ import java.io.Serializable; import java.util.Map; import java.util.Objects; - import javax.annotation.Nullable; import javax.jms.Destination; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java index 61a382d6ee2a..64dbad450d51 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java @@ -18,11 +18,9 @@ package org.apache.beam.sdk.io.kafka; import com.google.common.base.Joiner; - import java.io.IOException; import java.io.Serializable; import java.util.List; - import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.io.UnboundedSource; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java index e0e400eb5a8e..235fb1f8b2ae 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java @@ -19,7 +19,6 @@ import java.io.Serializable; import java.util.Arrays; - import org.apache.beam.sdk.values.KV; /** diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/CoderBasedKafkaDeserializer.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/CoderBasedKafkaDeserializer.java index ca552fb054a4..a1655860b36a 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/CoderBasedKafkaDeserializer.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/CoderBasedKafkaDeserializer.java @@ -21,7 +21,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import java.util.Map; - import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/CoderBasedKafkaSerializer.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/CoderBasedKafkaSerializer.java index 1044d6fe1792..84b617ee8ac4 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/CoderBasedKafkaSerializer.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/CoderBasedKafkaSerializer.java @@ -22,7 +22,6 @@ import java.util.Map; import javax.annotation.Nullable; - import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantDeserializer.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantDeserializer.java index fe4749f9f82b..76ceed56d99e 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantDeserializer.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantDeserializer.java @@ -19,7 +19,6 @@ package org.apache.beam.sdk.io.kafka.serialization; import java.util.Map; - import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.LongDeserializer; import org.joda.time.Instant; diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantSerializer.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantSerializer.java index 8fa4429d373b..0d69b2033f28 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantSerializer.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantSerializer.java @@ -19,7 +19,6 @@ package org.apache.beam.sdk.io.kafka.serialization; import java.util.Map; - import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; import org.joda.time.Instant; diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java index 0868ed443c84..b63775da7827 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java @@ -30,7 +30,6 @@ import com.mongodb.gridfs.GridFSDBFile; import com.mongodb.gridfs.GridFSInputFile; import com.mongodb.util.JSON; - import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; @@ -40,9 +39,7 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; - import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java index 8e7f03b0973f..826af1cc59a7 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java @@ -25,7 +25,6 @@ import com.mongodb.gridfs.GridFS; import com.mongodb.gridfs.GridFSDBFile; import com.mongodb.gridfs.GridFSInputFile; - import de.flapdoodle.embed.mongo.MongodExecutable; import de.flapdoodle.embed.mongo.MongodProcess; import de.flapdoodle.embed.mongo.MongodStarter; @@ -37,7 +36,6 @@ import de.flapdoodle.embed.mongo.distribution.Version; import de.flapdoodle.embed.process.io.file.Files; import de.flapdoodle.embed.process.runtime.Network; - import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -54,7 +52,6 @@ import java.util.List; import java.util.Random; import java.util.Scanner; - import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java index c36b7c8d344f..454c6ba1ea7b 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java @@ -23,7 +23,6 @@ import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoCursor; import com.mongodb.client.MongoDatabase; - import de.flapdoodle.embed.mongo.MongodExecutable; import de.flapdoodle.embed.mongo.MongodProcess; import de.flapdoodle.embed.mongo.MongodStarter; @@ -35,12 +34,10 @@ import de.flapdoodle.embed.mongo.distribution.Version; import de.flapdoodle.embed.process.io.file.Files; import de.flapdoodle.embed.process.runtime.Network; - import java.io.File; import java.io.Serializable; import java.net.ServerSocket; import java.util.ArrayList; - import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; @@ -50,7 +47,6 @@ import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.bson.Document; import org.junit.After; import org.junit.Assert; diff --git a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java index 0f25b0fb8357..228a85d77ab2 100644 --- a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java +++ b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.auto.value.AutoValue; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -28,9 +27,7 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.UUID; - import javax.annotation.Nullable; - import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; From b2553caf1350eaea3caefe55d5af414694c96424 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 3 May 2017 20:19:51 -0700 Subject: [PATCH 058/387] Move ValueWithRecordId to sdk.values, annotated --- .../org/apache/beam/runners/dataflow/DataflowRunner.java | 2 +- .../org/apache/beam/runners/spark/TestSparkRunner.java | 2 +- .../beam/sdk/io/BoundedReadFromUnboundedSource.java | 2 +- .../beam/sdk/{util => values}/ValueWithRecordId.java | 8 ++++++-- .../beam/sdk/{util => values}/ValueWithRecordIdTest.java | 4 ++-- 5 files changed, 11 insertions(+), 7 deletions(-) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => values}/ValueWithRecordId.java (93%) rename sdks/java/core/src/test/java/org/apache/beam/sdk/{util => values}/ValueWithRecordIdTest.java (92%) 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 7123316f436e..57da61b905b7 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 @@ -121,7 +121,6 @@ import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.NameUtils; import org.apache.beam.sdk.util.ReleaseInfo; -import org.apache.beam.sdk.util.ValueWithRecordId; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; @@ -132,6 +131,7 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index 6808d7b6b132..a6851c49ded5 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -48,11 +48,11 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.ValueWithRecordId; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.commons.io.FileUtils; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java index e54176fec0f5..d9adf9297db5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java @@ -37,9 +37,9 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.NameUtils; -import org.apache.beam.sdk.util.ValueWithRecordId; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.ValueWithRecordId; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java similarity index 93% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java index 9902aa7a1fb4..0d92f4075d55 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueWithRecordId.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.values; import com.google.common.base.MoreObjects; import java.io.IOException; @@ -24,16 +24,20 @@ import java.util.Arrays; import java.util.List; import java.util.Objects; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.transforms.DoFn; /** - * Immutable struct containing a value as well as a unique id identifying the value. + * For internal use only; no backwards compatibility guarantees. + * + *

    Immutable struct containing a value as well as a unique id identifying the value. * * @param the underlying value type */ +@Internal public class ValueWithRecordId { private final ValueT value; private final byte[] id; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ValueWithRecordIdTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/ValueWithRecordIdTest.java similarity index 92% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/ValueWithRecordIdTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/values/ValueWithRecordIdTest.java index e3a2dc6df80b..987c9af3baff 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ValueWithRecordIdTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/ValueWithRecordIdTest.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.values; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.ValueWithRecordId.ValueWithRecordIdCoder; +import org.apache.beam.sdk.values.ValueWithRecordId.ValueWithRecordIdCoder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; From c83cc744a69f735ac134471705e3403b9d5edd34 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 3 May 2017 20:34:54 -0700 Subject: [PATCH 059/387] Move some PCollectionView bits out of util --- .../org/apache/beam/runners/apex/ApexRunner.java | 2 +- .../construction/PTransformMatchersTest.java | 2 +- .../runners/direct}/PCollectionViewWindow.java | 2 +- .../beam/runners/direct/SideInputContainer.java | 1 - .../runners/direct/SideInputContainerTest.java | 2 +- .../runners/direct/ViewEvaluatorFactoryTest.java | 2 +- .../runners/direct/ViewOverrideFactoryTest.java | 2 +- .../direct/WriteWithShardingFactoryTest.java | 2 +- .../flink/FlinkStreamingViewOverrides.java | 2 +- .../runners/dataflow/BatchViewOverrides.java | 2 +- .../org/apache/beam/sdk/transforms/Combine.java | 2 +- .../org/apache/beam/sdk/transforms/View.java | 2 +- .../sdk/{util => values}/PCollectionViews.java | 16 +++++++--------- .../beam/sdk/transforms/DoFnTesterTest.java | 2 +- .../beam/sdk/io/gcp/bigquery/BigQueryIOTest.java | 2 +- 15 files changed, 20 insertions(+), 23 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => runners/direct-java/src/main/java/org/apache/beam/runners/direct}/PCollectionViewWindow.java (98%) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util => values}/PCollectionViews.java (97%) 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 1c845c6c628b..e1828c3217c6 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 @@ -60,9 +60,9 @@ 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.util.PCollectionViews; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; import org.apache.hadoop.conf.Configuration; /** diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java index cb28c3449295..6271234db021 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java @@ -60,13 +60,13 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.PCollectionViews; 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.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWindow.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java rename to runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWindow.java index 410c8cea6832..7a7d8ff238d7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWindow.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.runners.direct; import java.util.Objects; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java index 380dc65b0090..43da92ff916d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java @@ -39,7 +39,6 @@ import org.apache.beam.runners.core.SideInputReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.PCollectionViewWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java index d4ca9fdd76b7..5e7c79918514 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java @@ -45,11 +45,11 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; -import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java index 9560e94e0249..d8869b2ca25d 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java @@ -34,10 +34,10 @@ import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.View.CreatePCollectionView; import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.util.PCollectionViews; 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.PCollectionViews; import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java index a36787af86fc..eda00a785e0c 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java @@ -39,9 +39,9 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View.CreatePCollectionView; -import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; import org.junit.Rule; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java index 6fffd1a479ec..a2b0c5ce7733 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java @@ -54,9 +54,9 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java index f955f2a573ff..ce1c89555520 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingViewOverrides.java @@ -30,10 +30,10 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; /** * Flink streaming overrides for various view (side input) transforms. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index 1ff8a3f3affd..debaf59ef93e 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -74,7 +74,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; @@ -84,6 +83,7 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.WindowingStrategy; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index 1be948f779fb..666db3b80a65 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -57,13 +57,13 @@ import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.NameUtils; import org.apache.beam.sdk.util.NameUtils.NameOverride; -import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; 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 d17d423441a9..d7b81455b63f 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 @@ -25,10 +25,10 @@ import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; /** * Transforms for creating {@link PCollectionView PCollectionViews} from diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java index a07bc5e5818b..74887c71b368 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.values; import com.google.common.base.Function; import com.google.common.base.MoreObjects; @@ -33,6 +33,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; 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.coders.IterableCoder; import org.apache.beam.sdk.transforms.Materialization; @@ -41,18 +42,15 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.InvalidWindows; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PValueBase; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.WindowedValue; /** - * Implementations of {@link PCollectionView} shared across the SDK. + * For internal use only; no backwards compatibility guarantees. * - *

    For internal use only, subject to change. + *

    Implementations of {@link PCollectionView} shared across the SDK. */ +@Internal public class PCollectionViews { /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index f74d67344473..1bb71bbf1a18 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -34,10 +34,10 @@ 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.util.PCollectionViews; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; 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 38da2d94c112..a3b21eebd802 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 @@ -121,11 +121,11 @@ import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.MimeTypes; -import org.apache.beam.sdk.util.PCollectionViews; 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.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; From 02e80a7d5890ac2281d970cb1977399f79431d5b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 4 May 2017 16:09:26 -0700 Subject: [PATCH 060/387] Update Dataflow worker to 20170504-2 --- 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 0367bd8a95a1..e21b6de0f784 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -33,7 +33,7 @@ jar - beam-master-20170504 + beam-master-20170504-2 1 6 From a5faa413d58be7e0840f9aefbf746d9fdb4c2830 Mon Sep 17 00:00:00 2001 From: Sourabh Bajaj Date: Thu, 4 May 2017 15:26:07 -0700 Subject: [PATCH 061/387] [BEAM-2144] Remove transform.reflect and utils from Javadocs --- sdks/java/javadoc/ant.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/java/javadoc/ant.xml b/sdks/java/javadoc/ant.xml index 80dbdc2fe090..288f48bf9baa 100644 --- a/sdks/java/javadoc/ant.xml +++ b/sdks/java/javadoc/ant.xml @@ -79,7 +79,7 @@ offline="true" packageListLoc="joda-docs"/> - + @@ -89,6 +89,8 @@ + + From 040dbd4fd21a31dd3eb0f34fb9610ce857111db8 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 4 May 2017 17:01:24 -0700 Subject: [PATCH 062/387] Update expectation in ProcessBundleHandlerTest --- .../beam/fn/harness/control/ProcessBundleHandlerTest.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 3e06369320af..748ffea1f13c 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 @@ -18,6 +18,7 @@ package org.apache.beam.fn.harness.control; +import static org.apache.beam.sdk.util.WindowedValue.timestampedValueInGlobalWindow; import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.empty; @@ -433,7 +434,10 @@ public void testCreatingAndProcessingDoFn() throws Exception { additionalOutputValues.clear(); Iterables.getOnlyElement(finishFunctions).run(); - assertThat(mainOutputValues, contains(valueInGlobalWindow("FinishBundle"))); + assertThat( + mainOutputValues, + contains( + timestampedValueInGlobalWindow("FinishBundle", GlobalWindow.INSTANCE.maxTimestamp()))); mainOutputValues.clear(); } From c158e05ccdd1d06bd4f49c972288bf28c4e5d703 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Thu, 4 May 2017 16:14:34 -0700 Subject: [PATCH 063/387] [BEAM-1871] Move out test only utilities used only in examples/java out of sdks/java/core. --- examples/java/pom.xml | 12 ++++++++++++ .../apache/beam/examples/WindowedWordCountIT.java | 8 ++++---- .../java/org/apache/beam/examples/WordCountIT.java | 2 +- .../beam/examples/testing}/ExplicitShardedFile.java | 3 ++- .../beam/examples}/testing/FileChecksumMatcher.java | 5 ++--- .../examples}/testing/FileChecksumMatcherTest.java | 3 ++- .../beam/examples/testing}/NumberedShardedFile.java | 3 ++- .../examples/testing}/NumberedShardedFileTest.java | 2 +- .../apache/beam/examples/testing}/ShardedFile.java | 2 +- .../apache/beam/sdk/coders/StructuralByteArray.java | 5 ++--- .../apache/beam/sdk/testing/MatcherDeserializer.java | 4 ++-- .../apache/beam/sdk/testing/MatcherSerializer.java | 4 ++-- .../java/org/apache/beam/sdk/util/CoderUtils.java | 8 +++++--- 13 files changed, 38 insertions(+), 23 deletions(-) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => examples/java/src/test/java/org/apache/beam/examples/testing}/ExplicitShardedFile.java (98%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk => examples/java/src/test/java/org/apache/beam/examples}/testing/FileChecksumMatcher.java (97%) rename {sdks/java/core/src/test/java/org/apache/beam/sdk => examples/java/src/test/java/org/apache/beam/examples}/testing/FileChecksumMatcherTest.java (98%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => examples/java/src/test/java/org/apache/beam/examples/testing}/NumberedShardedFile.java (98%) rename {sdks/java/core/src/test/java/org/apache/beam/sdk/util => examples/java/src/test/java/org/apache/beam/examples/testing}/NumberedShardedFileTest.java (99%) rename {sdks/java/core/src/main/java/org/apache/beam/sdk/util => examples/java/src/test/java/org/apache/beam/examples/testing}/ShardedFile.java (97%) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index d673da27fe29..09473cd48827 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -491,6 +491,11 @@ google-auth-library-credentials + + com.google.code.findbugs + jsr305 + + org.apache.avro avro @@ -552,6 +557,13 @@ For testing the example itself, use the direct runner. This is separate from the use of ValidatesRunner tests for testing a particular runner. --> + + org.apache.beam + beam-sdks-java-core + tests + test + + org.apache.beam beam-runners-direct-java diff --git a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java index b5eddb5d17e7..01bc4027f814 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java @@ -33,21 +33,21 @@ import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.examples.common.WriteOneFilePerWindow.PerWindowFiles; +import org.apache.beam.examples.testing.ExplicitShardedFile; +import org.apache.beam.examples.testing.FileChecksumMatcher; +import org.apache.beam.examples.testing.NumberedShardedFile; +import org.apache.beam.examples.testing.ShardedFile; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; -import org.apache.beam.sdk.testing.FileChecksumMatcher; import org.apache.beam.sdk.testing.SerializableMatcher; import org.apache.beam.sdk.testing.StreamingIT; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.ExplicitShardedFile; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.NumberedShardedFile; -import org.apache.beam.sdk.util.ShardedFile; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; import org.joda.time.Duration; diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index 1660b61d1672..236ca9cb0780 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -20,10 +20,10 @@ import java.util.Date; import org.apache.beam.examples.WordCount.WordCountOptions; +import org.apache.beam.examples.testing.FileChecksumMatcher; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.FileChecksumMatcher; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.junit.BeforeClass; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java b/examples/java/src/test/java/org/apache/beam/examples/testing/ExplicitShardedFile.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/ExplicitShardedFile.java index 0f184de7f8f4..1dc7a62d07f2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/ExplicitShardedFile.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.examples.testing; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -35,6 +35,7 @@ import java.util.List; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; +import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java b/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcher.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcher.java index 82a6b7117636..8a0af1199f1e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcher.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.sdk.testing; +package org.apache.beam.examples.testing; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -31,9 +31,8 @@ import java.util.regex.Pattern; import javax.annotation.Nonnull; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.testing.SerializableMatcher; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.NumberedShardedFile; -import org.apache.beam.sdk.util.ShardedFile; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; import org.joda.time.Duration; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java b/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcherTest.java similarity index 98% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcherTest.java index 4ee67501817c..4d6eb6bbef49 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcherTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.testing; +package org.apache.beam.examples.testing; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; @@ -27,6 +27,7 @@ import java.nio.charset.StandardCharsets; import java.util.regex.Pattern; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java b/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFile.java similarity index 98% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFile.java index e18dd96f0046..f0b9c2d6045f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFile.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.examples.testing; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -44,6 +44,7 @@ import javax.annotation.Nonnull; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; +import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java b/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFileTest.java similarity index 99% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFileTest.java index 43a91665c9c3..83b8a4f136d4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFileTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.examples.testing; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java b/examples/java/src/test/java/org/apache/beam/examples/testing/ShardedFile.java similarity index 97% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java rename to examples/java/src/test/java/org/apache/beam/examples/testing/ShardedFile.java index ec9ed641976c..cd9537cb326d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java +++ b/examples/java/src/test/java/org/apache/beam/examples/testing/ShardedFile.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.examples.testing; import com.google.api.client.util.BackOff; import com.google.api.client.util.Sleeper; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java index 226f79cd429d..0ab0dea9bdab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java @@ -17,8 +17,7 @@ */ package org.apache.beam.sdk.coders; -import static com.google.api.client.util.Base64.encodeBase64String; - +import com.google.common.io.BaseEncoding; import java.util.Arrays; /** @@ -53,6 +52,6 @@ public int hashCode() { @Override public String toString() { - return "base64:" + encodeBase64String(value); + return "base64:" + BaseEncoding.base64().encode(value); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java index 6ca07badac92..e7aa5a7f4283 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.api.client.util.Base64; +import com.google.common.io.BaseEncoding; import java.io.IOException; import org.apache.beam.sdk.util.SerializableUtils; @@ -36,7 +36,7 @@ public SerializableMatcher deserialize(JsonParser jsonParser, throws IOException, JsonProcessingException { ObjectNode node = jsonParser.readValueAsTree(); String matcher = node.get("matcher").asText(); - byte[] in = Base64.decodeBase64(matcher); + byte[] in = BaseEncoding.base64().decode(matcher); return (SerializableMatcher) SerializableUtils .deserializeFromByteArray(in, "SerializableMatcher"); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java index 2b4584c8cdac..35375f66047d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.SerializerProvider; -import com.google.api.client.util.Base64; +import com.google.common.io.BaseEncoding; import java.io.IOException; import org.apache.beam.sdk.util.SerializableUtils; @@ -33,7 +33,7 @@ class MatcherSerializer extends JsonSerializer> { public void serialize(SerializableMatcher matcher, JsonGenerator jsonGenerator, SerializerProvider serializerProvider) throws IOException, JsonProcessingException { byte[] out = SerializableUtils.serializeToByteArray(matcher); - String encodedString = Base64.encodeBase64String(out); + String encodedString = BaseEncoding.base64().encode(out); jsonGenerator.writeStartObject(); jsonGenerator.writeStringField("matcher", encodedString); jsonGenerator.writeEndObject(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java index 3380a102e68a..e3ae485f5d18 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.util; -import com.google.api.client.util.Base64; import com.google.common.base.Throwables; +import com.google.common.io.BaseEncoding; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -163,7 +163,7 @@ public static T clone(Coder coder, T value) throws CoderException { public static String encodeToBase64(Coder coder, T value) throws CoderException { byte[] rawValue = encodeToByteArray(coder, value); - return Base64.encodeBase64URLSafeString(rawValue); + return BaseEncoding.base64Url().omitPadding().encode(rawValue); } /** @@ -171,7 +171,9 @@ public static String encodeToBase64(Coder coder, T value) */ public static T decodeFromBase64(Coder coder, String encodedValue) throws CoderException { return decodeFromSafeStream( - coder, new ByteArrayInputStream(Base64.decodeBase64(encodedValue)), Coder.Context.OUTER); + coder, + new ByteArrayInputStream(BaseEncoding.base64Url().omitPadding().decode(encodedValue)), + Coder.Context.OUTER); } /** From b130d7aac466860ed5a7abec0bef33d9e0dd3c6d Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 18:14:13 -0700 Subject: [PATCH 064/387] Revert "This closes #2905" I misread Jenkins and should not have merged. I am sorry. This reverts commit 9f27c33ec7e7c61afbca0395f932275b354eb428, reversing changes made to 5fc3d335919207c23bc6fd2047e9e38351754ff1. --- examples/java/pom.xml | 12 ------------ .../apache/beam/examples/WindowedWordCountIT.java | 8 ++++---- .../java/org/apache/beam/examples/WordCountIT.java | 2 +- .../apache/beam/sdk/coders/StructuralByteArray.java | 5 +++-- .../beam/sdk}/testing/FileChecksumMatcher.java | 5 +++-- .../apache/beam/sdk/testing/MatcherDeserializer.java | 4 ++-- .../apache/beam/sdk/testing/MatcherSerializer.java | 4 ++-- .../java/org/apache/beam/sdk/util/CoderUtils.java | 8 +++----- .../apache/beam/sdk/util}/ExplicitShardedFile.java | 3 +-- .../apache/beam/sdk/util}/NumberedShardedFile.java | 3 +-- .../java/org/apache/beam/sdk/util}/ShardedFile.java | 2 +- .../beam/sdk}/testing/FileChecksumMatcherTest.java | 3 +-- .../beam/sdk/util}/NumberedShardedFileTest.java | 2 +- 13 files changed, 23 insertions(+), 38 deletions(-) rename {examples/java/src/test/java/org/apache/beam/examples => sdks/java/core/src/main/java/org/apache/beam/sdk}/testing/FileChecksumMatcher.java (97%) rename {examples/java/src/test/java/org/apache/beam/examples/testing => sdks/java/core/src/main/java/org/apache/beam/sdk/util}/ExplicitShardedFile.java (98%) rename {examples/java/src/test/java/org/apache/beam/examples/testing => sdks/java/core/src/main/java/org/apache/beam/sdk/util}/NumberedShardedFile.java (98%) rename {examples/java/src/test/java/org/apache/beam/examples/testing => sdks/java/core/src/main/java/org/apache/beam/sdk/util}/ShardedFile.java (97%) rename {examples/java/src/test/java/org/apache/beam/examples => sdks/java/core/src/test/java/org/apache/beam/sdk}/testing/FileChecksumMatcherTest.java (98%) rename {examples/java/src/test/java/org/apache/beam/examples/testing => sdks/java/core/src/test/java/org/apache/beam/sdk/util}/NumberedShardedFileTest.java (99%) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 09473cd48827..d673da27fe29 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -491,11 +491,6 @@ google-auth-library-credentials - - com.google.code.findbugs - jsr305 - - org.apache.avro avro @@ -557,13 +552,6 @@ For testing the example itself, use the direct runner. This is separate from the use of ValidatesRunner tests for testing a particular runner. --> - - org.apache.beam - beam-sdks-java-core - tests - test - - org.apache.beam beam-runners-direct-java diff --git a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java index 01bc4027f814..b5eddb5d17e7 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java @@ -33,21 +33,21 @@ import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.examples.common.WriteOneFilePerWindow.PerWindowFiles; -import org.apache.beam.examples.testing.ExplicitShardedFile; -import org.apache.beam.examples.testing.FileChecksumMatcher; -import org.apache.beam.examples.testing.NumberedShardedFile; -import org.apache.beam.examples.testing.ShardedFile; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; +import org.apache.beam.sdk.testing.FileChecksumMatcher; import org.apache.beam.sdk.testing.SerializableMatcher; import org.apache.beam.sdk.testing.StreamingIT; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.util.ExplicitShardedFile; import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.NumberedShardedFile; +import org.apache.beam.sdk.util.ShardedFile; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; import org.joda.time.Duration; diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index 236ca9cb0780..1660b61d1672 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -20,10 +20,10 @@ import java.util.Date; import org.apache.beam.examples.WordCount.WordCountOptions; -import org.apache.beam.examples.testing.FileChecksumMatcher; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.FileChecksumMatcher; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.junit.BeforeClass; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java index 0ab0dea9bdab..226f79cd429d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java @@ -17,7 +17,8 @@ */ package org.apache.beam.sdk.coders; -import com.google.common.io.BaseEncoding; +import static com.google.api.client.util.Base64.encodeBase64String; + import java.util.Arrays; /** @@ -52,6 +53,6 @@ public int hashCode() { @Override public String toString() { - return "base64:" + BaseEncoding.base64().encode(value); + return "base64:" + encodeBase64String(value); } } diff --git a/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java similarity index 97% rename from examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcher.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java index 8a0af1199f1e..82a6b7117636 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcher.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.examples.testing; +package org.apache.beam.sdk.testing; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -31,8 +31,9 @@ import java.util.regex.Pattern; import javax.annotation.Nonnull; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.testing.SerializableMatcher; import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.NumberedShardedFile; +import org.apache.beam.sdk.util.ShardedFile; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; import org.joda.time.Duration; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java index e7aa5a7f4283..6ca07badac92 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.common.io.BaseEncoding; +import com.google.api.client.util.Base64; import java.io.IOException; import org.apache.beam.sdk.util.SerializableUtils; @@ -36,7 +36,7 @@ public SerializableMatcher deserialize(JsonParser jsonParser, throws IOException, JsonProcessingException { ObjectNode node = jsonParser.readValueAsTree(); String matcher = node.get("matcher").asText(); - byte[] in = BaseEncoding.base64().decode(matcher); + byte[] in = Base64.decodeBase64(matcher); return (SerializableMatcher) SerializableUtils .deserializeFromByteArray(in, "SerializableMatcher"); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java index 35375f66047d..2b4584c8cdac 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.SerializerProvider; -import com.google.common.io.BaseEncoding; +import com.google.api.client.util.Base64; import java.io.IOException; import org.apache.beam.sdk.util.SerializableUtils; @@ -33,7 +33,7 @@ class MatcherSerializer extends JsonSerializer> { public void serialize(SerializableMatcher matcher, JsonGenerator jsonGenerator, SerializerProvider serializerProvider) throws IOException, JsonProcessingException { byte[] out = SerializableUtils.serializeToByteArray(matcher); - String encodedString = BaseEncoding.base64().encode(out); + String encodedString = Base64.encodeBase64String(out); jsonGenerator.writeStartObject(); jsonGenerator.writeStringField("matcher", encodedString); jsonGenerator.writeEndObject(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java index e3ae485f5d18..3380a102e68a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.util; +import com.google.api.client.util.Base64; import com.google.common.base.Throwables; -import com.google.common.io.BaseEncoding; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -163,7 +163,7 @@ public static T clone(Coder coder, T value) throws CoderException { public static String encodeToBase64(Coder coder, T value) throws CoderException { byte[] rawValue = encodeToByteArray(coder, value); - return BaseEncoding.base64Url().omitPadding().encode(rawValue); + return Base64.encodeBase64URLSafeString(rawValue); } /** @@ -171,9 +171,7 @@ public static String encodeToBase64(Coder coder, T value) */ public static T decodeFromBase64(Coder coder, String encodedValue) throws CoderException { return decodeFromSafeStream( - coder, - new ByteArrayInputStream(BaseEncoding.base64Url().omitPadding().decode(encodedValue)), - Coder.Context.OUTER); + coder, new ByteArrayInputStream(Base64.decodeBase64(encodedValue)), Coder.Context.OUTER); } /** diff --git a/examples/java/src/test/java/org/apache/beam/examples/testing/ExplicitShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java similarity index 98% rename from examples/java/src/test/java/org/apache/beam/examples/testing/ExplicitShardedFile.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java index 1dc7a62d07f2..0f184de7f8f4 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/testing/ExplicitShardedFile.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExplicitShardedFile.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.examples.testing; +package org.apache.beam.sdk.util; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -35,7 +35,6 @@ import java.util.List; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; -import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java similarity index 98% rename from examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFile.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java index f0b9c2d6045f..e18dd96f0046 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFile.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.examples.testing; +package org.apache.beam.sdk.util; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -44,7 +44,6 @@ import javax.annotation.Nonnull; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; -import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/examples/java/src/test/java/org/apache/beam/examples/testing/ShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java similarity index 97% rename from examples/java/src/test/java/org/apache/beam/examples/testing/ShardedFile.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java index cd9537cb326d..ec9ed641976c 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/testing/ShardedFile.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedFile.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.beam.examples.testing; +package org.apache.beam.sdk.util; import com.google.api.client.util.BackOff; import com.google.api.client.util.Sleeper; diff --git a/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java similarity index 98% rename from examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcherTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java index 4d6eb6bbef49..4ee67501817c 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/testing/FileChecksumMatcherTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.testing; +package org.apache.beam.sdk.testing; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; @@ -27,7 +27,6 @@ import java.nio.charset.StandardCharsets; import java.util.regex.Pattern; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFileTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java similarity index 99% rename from examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFileTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java index 83b8a4f136d4..43a91665c9c3 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/testing/NumberedShardedFileTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.testing; +package org.apache.beam.sdk.util; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; From 6be71c9a1c2ec43478e2ddaedaba67608de129ae Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 4 May 2017 19:03:50 -0700 Subject: [PATCH 065/387] Adjust error message expectation in DoFnSignaturesTest --- .../beam/sdk/transforms/reflect/DoFnSignaturesTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java index 5b97a553c28d..cffb0adf613a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java @@ -385,7 +385,7 @@ public void testTimerIdDuplicate() throws Exception { thrown.expectMessage("my-id"); thrown.expectMessage("myfield1"); thrown.expectMessage("myfield2"); - thrown.expectMessage(not(mentionsState())); + thrown.expectMessage(not(containsString("State"))); // lowercase "state" is in the package name thrown.expectMessage(mentionsTimers()); DoFnSignature sig = DoFnSignatures.getSignature( @@ -407,7 +407,7 @@ public void testTimerIdNonFinal() throws Exception { thrown.expectMessage("Timer declarations must be final"); thrown.expectMessage("Non-final field"); thrown.expectMessage("myfield"); - thrown.expectMessage(not(mentionsState())); + thrown.expectMessage(not(containsString("State"))); // lowercase "state" is in the package name thrown.expectMessage(mentionsTimers()); DoFnSignature sig = DoFnSignatures.getSignature( From c1811a4adbfb290226d183eac01bbe9bdbce2462 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 4 May 2017 21:34:54 -0700 Subject: [PATCH 066/387] No parallelism for Apex WindowedWordCountIT --- examples/java/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 65625df297b3..fb2613514e8c 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -305,7 +305,7 @@ WordCountIT.java WindowedWordCountIT.java - all + none 4 From fc4534cd6e5366a5f12cefebcd52ac1fe7cdde41 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Tue, 4 Apr 2017 18:28:15 +0800 Subject: [PATCH 067/387] [BEAM-1862] SplittableDoFnOperator should close the ScheduledExecutorService --- .../streaming/SplittableDoFnOperator.java | 28 ++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) 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 7d54cfa51d5d..968fc0a540a7 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 @@ -24,6 +24,8 @@ import java.util.List; import java.util.Map; 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; @@ -57,6 +59,8 @@ public class SplittableDoFnOperator< extends DoFnOperator< KeyedWorkItem>, FnOutputT, OutputT> { + private transient ScheduledExecutorService executorService; + public SplittableDoFnOperator( DoFn>, FnOutputT> doFn, String stepName, @@ -108,6 +112,8 @@ public TimerInternals timerInternalsForKey(String key) { } }; + executorService = Executors.newSingleThreadScheduledExecutor(Executors.defaultThreadFactory()); + ((SplittableParDo.ProcessFn) doFn).setStateInternalsFactory(stateInternalsFactory); ((SplittableParDo.ProcessFn) doFn).setTimerInternalsFactory(timerInternalsFactory); ((SplittableParDo.ProcessFn) doFn).setProcessElementInvoker( @@ -137,7 +143,7 @@ public void outputWindowedValue( } }, sideInputReader, - Executors.newSingleThreadScheduledExecutor(Executors.defaultThreadFactory()), + executorService, 10000, Duration.standardSeconds(10))); } @@ -149,4 +155,24 @@ public void fireTimer(InternalTimer timer) { (String) stateInternals.getKey(), Collections.singletonList(timer.getNamespace())))); } + + @Override + public void close() throws Exception { + super.close(); + + executorService.shutdown(); + + long shutdownTimeout = Duration.standardSeconds(10).getMillis(); + try { + if (!executorService.awaitTermination(shutdownTimeout, TimeUnit.MILLISECONDS)) { + LOG.debug("The scheduled executor service did not properly terminate. Shutting " + + "it down now."); + executorService.shutdownNow(); + } + } catch (InterruptedException e) { + LOG.debug("Could not properly await the termination of the scheduled executor service.", e); + executorService.shutdownNow(); + } + } + } From 0066ca8d9bca3a5e2106548f2736351913c7f681 Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Tue, 2 May 2017 08:21:45 -0700 Subject: [PATCH 068/387] [BEAM-2180] update Apex version to 3.6.0 --- runners/apex/pom.xml | 4 ++-- .../java/org/apache/beam/runners/apex/ApexRunnerTest.java | 2 +- runners/apex/src/test/resources/beam-runners-apex.properties | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index aa4bddf57f76..85b15fa688d9 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -33,7 +33,7 @@ jar - 3.5.0 + 3.6.0 3.4.0 true @@ -243,7 +243,7 @@ - org.apache.apex:apex-api:jar:3.5.0 + org.apache.apex:apex-api:jar:${apex.core.version} org.apache.commons:commons-lang3::3.1 commons-io:commons-io:jar:2.4 com.esotericsoftware.kryo:kryo::${apex.kryo.version} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexRunnerTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexRunnerTest.java index 0fedc47fd055..e9e9a5b5c6a8 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexRunnerTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/ApexRunnerTest.java @@ -58,7 +58,7 @@ public void testConfigProperties() throws Exception { File tmp = File.createTempFile("beam-runners-apex-", ".properties"); tmp.deleteOnExit(); Properties props = new Properties(); - props.setProperty("dt.operator." + operName + ".attr.MEMORY_MB", "64"); + props.setProperty("apex.operator." + operName + ".attr.MEMORY_MB", "64"); try (FileOutputStream fos = new FileOutputStream(tmp)) { props.store(fos, ""); } diff --git a/runners/apex/src/test/resources/beam-runners-apex.properties b/runners/apex/src/test/resources/beam-runners-apex.properties index 48f8b05959b8..55cb397390c7 100644 --- a/runners/apex/src/test/resources/beam-runners-apex.properties +++ b/runners/apex/src/test/resources/beam-runners-apex.properties @@ -17,4 +17,4 @@ ################################################################################ # properties for unit test -dt.operator.testProperties.attr.MEMORY_MB=32 +apex.operator.testProperties.attr.MEMORY_MB=32 From 7512a73cf8aa2a527c89ecb054e92207411ed241 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Thu, 4 May 2017 18:33:16 -0700 Subject: [PATCH 069/387] [BEAM-2016] Delete HdfsFileSource & Sink --- sdks/java/io/hadoop-file-system/README.md | 43 -- sdks/java/io/hadoop-file-system/pom.xml | 24 - .../apache/beam/sdk/io/hdfs/HDFSFileSink.java | 478 -------------- .../beam/sdk/io/hdfs/HDFSFileSource.java | 625 ------------------ .../org/apache/beam/sdk/io/hdfs/Sink.java | 195 ------ .../apache/beam/sdk/io/hdfs/UGIHelper.java | 38 -- .../org/apache/beam/sdk/io/hdfs/Write.java | 588 ---------------- .../apache/beam/sdk/io/hdfs/package-info.java | 3 +- .../beam/sdk/io/hdfs/HDFSFileSinkTest.java | 172 ----- .../beam/sdk/io/hdfs/HDFSFileSourceTest.java | 231 ------- 10 files changed, 2 insertions(+), 2395 deletions(-) delete mode 100644 sdks/java/io/hadoop-file-system/README.md delete mode 100644 sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java delete mode 100644 sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java delete mode 100644 sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Sink.java delete mode 100644 sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/UGIHelper.java delete mode 100644 sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Write.java delete mode 100644 sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSinkTest.java delete mode 100644 sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java diff --git a/sdks/java/io/hadoop-file-system/README.md b/sdks/java/io/hadoop-file-system/README.md deleted file mode 100644 index 3a734f214ae7..000000000000 --- a/sdks/java/io/hadoop-file-system/README.md +++ /dev/null @@ -1,43 +0,0 @@ - - -# HDFS IO - -This library provides HDFS sources and sinks to make it possible to read and -write Apache Hadoop file formats from Apache Beam pipelines. - -Currently, only the read path is implemented. A `HDFSFileSource` allows any -Hadoop `FileInputFormat` to be read as a `PCollection`. - -A `HDFSFileSource` can be read from using the -`org.apache.beam.sdk.io.Read` transform. For example: - -```java -HDFSFileSource source = HDFSFileSource.from(path, MyInputFormat.class, - MyKey.class, MyValue.class); -PCollection> records = pipeline.apply(Read.from(mySource)); -``` - -Alternatively, the `readFrom` method is a convenience method that returns a read -transform. For example: - -```java -PCollection> records = pipeline.apply(HDFSFileSource.readFrom(path, - MyInputFormat.class, MyKey.class, MyValue.class)); -``` diff --git a/sdks/java/io/hadoop-file-system/pom.xml b/sdks/java/io/hadoop-file-system/pom.xml index 562277eeb116..3b392c28b27c 100644 --- a/sdks/java/io/hadoop-file-system/pom.xml +++ b/sdks/java/io/hadoop-file-system/pom.xml @@ -81,11 +81,6 @@ beam-sdks-java-core - - org.apache.beam - beam-sdks-java-io-hadoop-common - - com.fasterxml.jackson.core jackson-core @@ -123,25 +118,6 @@ jsr305 - - org.apache.avro - avro - - - - org.apache.avro - avro-mapred - ${avro.version} - hadoop2 - - - - org.mortbay.jetty - servlet-api - - - - org.apache.hadoop hadoop-client diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java deleted file mode 100644 index aee73c4f0561..000000000000 --- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java +++ /dev/null @@ -1,478 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.hdfs; - -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; - -import com.google.auto.value.AutoValue; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import java.io.IOException; -import java.net.URI; -import java.security.PrivilegedExceptionAction; -import java.util.Random; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.mapred.AvroKey; -import org.apache.avro.mapreduce.AvroKeyOutputFormat; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.hadoop.SerializableConfiguration; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.values.KV; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.JobID; -import org.apache.hadoop.mapreduce.RecordWriter; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.mapreduce.TaskID; -import org.apache.hadoop.mapreduce.TaskType; -import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; -import org.apache.hadoop.mapreduce.task.JobContextImpl; -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; - -/** - * A {@link Sink} for writing records to a Hadoop filesystem using a Hadoop file-based - * output - * format. - * - *

    To write a {@link org.apache.beam.sdk.values.PCollection} of elements of type T to Hadoop - * filesystem use {@link HDFSFileSink#to}, specify the path (this can be any Hadoop supported - * filesystem: HDFS, S3, GCS etc), the Hadoop {@link FileOutputFormat}, the key class K and the - * value class V and finally the {@link SerializableFunction} to map from T to {@link KV} of K - * and V. - * - *

    {@code HDFSFileSink} can be used by {@link Write} to create write - * transform. See example below. - * - *

    {@code HDFSFileSink} comes with helper methods to write text and Apache Avro. For example: - * - *

    - * {@code
    - * HDFSFileSink, NullWritable> sink =
    - *   HDFSFileSink.toAvro(path, AvroCoder.of(CustomSpecificAvroClass.class));
    - * avroRecordsPCollection.apply(Write.to(sink));
    - * }
    - * 
    - * - * @param the type of elements of the input {@link org.apache.beam.sdk.values.PCollection}. - * @param the type of keys to be written to the sink via {@link FileOutputFormat}. - * @param the type of values to be written to the sink via {@link FileOutputFormat}. - */ -@AutoValue -@Experimental -public abstract class HDFSFileSink extends Sink { - - private static final JobID jobId = new JobID( - Long.toString(System.currentTimeMillis()), - new Random().nextInt(Integer.MAX_VALUE)); - - public abstract String path(); - public abstract Class> formatClass(); - public abstract Class keyClass(); - public abstract Class valueClass(); - public abstract SerializableFunction> outputConverter(); - public abstract SerializableConfiguration serializableConfiguration(); - public @Nullable abstract String username(); - public abstract boolean validate(); - - // ======================================================================= - // Factory methods - // ======================================================================= - - public static > HDFSFileSink - to(String path, - Class formatClass, - Class keyClass, - Class valueClass, - SerializableFunction> outputConverter) { - return HDFSFileSink.builder() - .setPath(path) - .setFormatClass(formatClass) - .setKeyClass(keyClass) - .setValueClass(valueClass) - .setOutputConverter(outputConverter) - .setConfiguration(null) - .setUsername(null) - .setValidate(true) - .build(); - } - - public static HDFSFileSink toText(String path) { - SerializableFunction> outputConverter = - new SerializableFunction>() { - @Override - public KV apply(T input) { - return KV.of(NullWritable.get(), new Text(input.toString())); - } - }; - return to(path, TextOutputFormat.class, NullWritable.class, Text.class, outputConverter); - } - - /** - * Helper to create Avro sink given {@link AvroCoder}. Keep in mind that configuration - * object is altered to enable Avro output. - */ - public static HDFSFileSink, NullWritable> toAvro(String path, - final AvroCoder coder, - Configuration conf) { - SerializableFunction, NullWritable>> outputConverter = - new SerializableFunction, NullWritable>>() { - @Override - public KV, NullWritable> apply(T input) { - return KV.of(new AvroKey<>(input), NullWritable.get()); - } - }; - conf.set("avro.schema.output.key", coder.getSchema().toString()); - return to( - path, - AvroKeyOutputFormat.class, - (Class>) (Class) AvroKey.class, - NullWritable.class, - outputConverter).withConfiguration(conf); - } - - /** - * Helper to create Avro sink given {@link Schema}. Keep in mind that configuration - * object is altered to enable Avro output. - */ - public static HDFSFileSink, NullWritable> - toAvro(String path, Schema schema, Configuration conf) { - return toAvro(path, AvroCoder.of(schema), conf); - } - - /** - * Helper to create Avro sink given {@link Class}. Keep in mind that configuration - * object is altered to enable Avro output. - */ - public static HDFSFileSink, NullWritable> toAvro(String path, - Class cls, - Configuration conf) { - return toAvro(path, AvroCoder.of(cls), conf); - } - - // ======================================================================= - // Builder methods - // ======================================================================= - - public abstract Builder toBuilder(); - public static Builder builder() { - return new AutoValue_HDFSFileSink.Builder<>(); - } - - /** - * AutoValue builder for {@link HDFSFileSink}. - */ - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setPath(String path); - public abstract Builder setFormatClass( - Class> formatClass); - public abstract Builder setKeyClass(Class keyClass); - public abstract Builder setValueClass(Class valueClass); - public abstract Builder setOutputConverter( - SerializableFunction> outputConverter); - public abstract Builder setSerializableConfiguration( - SerializableConfiguration serializableConfiguration); - public Builder setConfiguration(@Nullable Configuration configuration) { - if (configuration == null) { - configuration = new Configuration(false); - } - return this.setSerializableConfiguration(new SerializableConfiguration(configuration)); - } - public abstract Builder setUsername(String username); - public abstract Builder setValidate(boolean validate); - public abstract HDFSFileSink build(); - } - - public HDFSFileSink withConfiguration(@Nullable Configuration configuration) { - return this.toBuilder().setConfiguration(configuration).build(); - } - - public HDFSFileSink withUsername(@Nullable String username) { - return this.toBuilder().setUsername(username).build(); - } - - // ======================================================================= - // Sink - // ======================================================================= - - @Override - public void validate(PipelineOptions options) { - if (validate()) { - try { - UGIHelper.getBestUGI(username()).doAs(new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - FileSystem fs = FileSystem.get(new URI(path()), - SerializableConfiguration.newConfiguration(serializableConfiguration())); - checkState(!fs.exists(new Path(path())), "Output path %s already exists", path()); - return null; - } - }); - } catch (IOException | InterruptedException e) { - throw new RuntimeException(e); - } - } - } - - @Override - public Sink.WriteOperation createWriteOperation() { - return new HDFSWriteOperation<>(this, path(), formatClass()); - } - - private Job newJob() throws IOException { - Job job = SerializableConfiguration.newJob(serializableConfiguration()); - job.setJobID(jobId); - job.setOutputKeyClass(keyClass()); - job.setOutputValueClass(valueClass()); - return job; - } - - // ======================================================================= - // WriteOperation - // ======================================================================= - - /** {{@link WriteOperation}} for HDFS. */ - private static class HDFSWriteOperation extends WriteOperation { - - private final HDFSFileSink sink; - private final String path; - private final Class> formatClass; - - HDFSWriteOperation(HDFSFileSink sink, - String path, - Class> formatClass) { - this.sink = sink; - this.path = path; - this.formatClass = formatClass; - } - - @Override - public void initialize(PipelineOptions options) throws Exception { - Job job = sink.newJob(); - FileOutputFormat.setOutputPath(job, new Path(path)); - } - - @Override - public void setWindowedWrites(boolean windowedWrites) { - } - - @Override - public void finalize(final Iterable writerResults, PipelineOptions options) - throws Exception { - UGIHelper.getBestUGI(sink.username()).doAs(new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - doFinalize(writerResults); - return null; - } - }); - } - - private void doFinalize(Iterable writerResults) throws Exception { - Job job = sink.newJob(); - FileSystem fs = FileSystem.get(new URI(path), job.getConfiguration()); - // If there are 0 output shards, just create output folder. - if (!writerResults.iterator().hasNext()) { - fs.mkdirs(new Path(path)); - return; - } - - // job successful - JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID()); - FileOutputCommitter outputCommitter = new FileOutputCommitter(new Path(path), context); - outputCommitter.commitJob(context); - - // get actual output shards - Set actual = Sets.newHashSet(); - FileStatus[] statuses = fs.listStatus(new Path(path), new PathFilter() { - @Override - public boolean accept(Path path) { - String name = path.getName(); - return !name.startsWith("_") && !name.startsWith("."); - } - }); - - // get expected output shards - Set expected = Sets.newHashSet(writerResults); - checkState( - expected.size() == Lists.newArrayList(writerResults).size(), - "Data loss due to writer results hash collision"); - for (FileStatus s : statuses) { - String name = s.getPath().getName(); - int pos = name.indexOf('.'); - actual.add(pos > 0 ? name.substring(0, pos) : name); - } - - checkState(actual.equals(expected), "Writer results and output files do not match"); - - // rename output shards to Hadoop style, i.e. part-r-00000.txt - int i = 0; - for (FileStatus s : statuses) { - String name = s.getPath().getName(); - int pos = name.indexOf('.'); - String ext = pos > 0 ? name.substring(pos) : ""; - fs.rename( - s.getPath(), - new Path(s.getPath().getParent(), String.format("part-r-%05d%s", i, ext))); - i++; - } - } - - @Override - public Writer createWriter(PipelineOptions options) throws Exception { - return new HDFSWriter<>(this, path, formatClass); - } - - @Override - public Sink getSink() { - return sink; - } - - @Override - public Coder getWriterResultCoder() { - return StringUtf8Coder.of(); - } - - } - - // ======================================================================= - // Writer - // ======================================================================= - - private static class HDFSWriter extends Writer { - - private final HDFSWriteOperation writeOperation; - private final String path; - private final Class> formatClass; - - // unique hash for each task - private int hash; - - private TaskAttemptContext context; - private RecordWriter recordWriter; - private FileOutputCommitter outputCommitter; - - HDFSWriter(HDFSWriteOperation writeOperation, - String path, - Class> formatClass) { - this.writeOperation = writeOperation; - this.path = path; - this.formatClass = formatClass; - } - - @Override - public void openWindowed(final String uId, - BoundedWindow window, - PaneInfo paneInfo, - int shard, - int numShards) throws Exception { - throw new UnsupportedOperationException("Windowing support not implemented yet for" - + "HDFS. Window " + window); - } - - @Override - public void openUnwindowed(final String uId, int shard, int numShards) throws Exception { - UGIHelper.getBestUGI(writeOperation.sink.username()).doAs( - new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - doOpen(uId); - return null; - } - } - ); - } - - private void doOpen(String uId) throws Exception { - this.hash = uId.hashCode(); - - Job job = writeOperation.sink.newJob(); - FileOutputFormat.setOutputPath(job, new Path(path)); - - // Each Writer is responsible for writing one bundle of elements and is represented by one - // unique Hadoop task based on uId/hash. All tasks share the same job ID. - JobID jobId = job.getJobID(); - TaskID taskId = new TaskID(jobId, TaskType.REDUCE, hash); - context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID(taskId, 0)); - - FileOutputFormat outputFormat = formatClass.newInstance(); - recordWriter = outputFormat.getRecordWriter(context); - outputCommitter = (FileOutputCommitter) outputFormat.getOutputCommitter(context); - } - - @Override - public void write(T value) throws Exception { - checkNotNull(recordWriter, - "Record writer can't be null. Make sure to open Writer first!"); - KV kv = writeOperation.sink.outputConverter().apply(value); - recordWriter.write(kv.getKey(), kv.getValue()); - } - - @Override - public void cleanup() throws Exception { - - } - - @Override - public String close() throws Exception { - return UGIHelper.getBestUGI(writeOperation.sink.username()).doAs( - new PrivilegedExceptionAction() { - @Override - public String run() throws Exception { - return doClose(); - } - }); - } - - private String doClose() throws Exception { - // task/attempt successful - recordWriter.close(context); - outputCommitter.commitTask(context); - - // result is prefix of the output file name - return String.format("part-r-%d", hash); - } - - @Override - public WriteOperation getWriteOperation() { - return writeOperation; - } - - } - -} diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java deleted file mode 100644 index 5cc20978628a..000000000000 --- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java +++ /dev/null @@ -1,625 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.hdfs; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkState; - -import com.google.auto.value.AutoValue; -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.security.PrivilegedExceptionAction; -import java.util.List; -import java.util.ListIterator; -import java.util.NoSuchElementException; -import javax.annotation.Nullable; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.mapred.AvroKey; -import org.apache.avro.mapreduce.AvroKeyInputFormat; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VoidCoder; -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.SerializableFunction; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.values.KV; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableUtils; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A {@code BoundedSource} for reading files resident in a Hadoop filesystem using a - * Hadoop file-based input format. - * - *

    To read a {@link org.apache.beam.sdk.values.PCollection} of - * {@link org.apache.beam.sdk.values.KV} key-value pairs from one or more - * Hadoop files, use {@link HDFSFileSource#from} to specify the path(s) of the files to - * read, the Hadoop {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat}, the - * key class and the value class. - * - *

    A {@code HDFSFileSource} can be read from using the - * {@link org.apache.beam.sdk.io.Read} transform. For example: - * - *

    - * {@code
    - * HDFSFileSource source = HDFSFileSource.from(path, MyInputFormat.class,
    - *   MyKey.class, MyValue.class);
    - * PCollection> records = pipeline.apply(Read.from(mySource));
    - * }
    - * 
    - * - *

    Implementation note: Since Hadoop's - * {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat} - * determines the input splits, this class extends {@link BoundedSource} rather than - * {@link org.apache.beam.sdk.io.OffsetBasedSource}, since the latter - * dictates input splits. - * @param the type of elements of the result {@link org.apache.beam.sdk.values.PCollection}. - * @param the type of keys to be read from the source via {@link FileInputFormat}. - * @param the type of values to be read from the source via {@link FileInputFormat}. - */ -@AutoValue -@Experimental -public abstract class HDFSFileSource extends BoundedSource { - private static final long serialVersionUID = 0L; - - private static final Logger LOG = LoggerFactory.getLogger(HDFSFileSource.class); - - public abstract String filepattern(); - public abstract Class> formatClass(); - public abstract Coder coder(); - public abstract SerializableFunction, T> inputConverter(); - public abstract SerializableConfiguration serializableConfiguration(); - public @Nullable abstract SerializableSplit serializableSplit(); - public @Nullable abstract String username(); - public abstract boolean validateSource(); - - // ======================================================================= - // Factory methods - // ======================================================================= - - public static > HDFSFileSource - from(String filepattern, - Class formatClass, - Coder coder, - SerializableFunction, T> inputConverter) { - return HDFSFileSource.builder() - .setFilepattern(filepattern) - .setFormatClass(formatClass) - .setCoder(coder) - .setInputConverter(inputConverter) - .setConfiguration(null) - .setUsername(null) - .setValidateSource(true) - .setSerializableSplit(null) - .build(); - } - - public static > HDFSFileSource, K, V> - from(String filepattern, - Class formatClass, - Class keyClass, - Class valueClass) { - KvCoder coder = KvCoder.of(getDefaultCoder(keyClass), getDefaultCoder(valueClass)); - SerializableFunction, KV> inputConverter = - new SerializableFunction, KV>() { - @Override - public KV apply(KV input) { - return input; - } - }; - return HDFSFileSource., K, V>builder() - .setFilepattern(filepattern) - .setFormatClass(formatClass) - .setCoder(coder) - .setInputConverter(inputConverter) - .setConfiguration(null) - .setUsername(null) - .setValidateSource(true) - .setSerializableSplit(null) - .build(); - } - - public static HDFSFileSource - fromText(String filepattern) { - SerializableFunction, String> inputConverter = - new SerializableFunction, String>() { - @Override - public String apply(KV input) { - return input.getValue().toString(); - } - }; - return from(filepattern, TextInputFormat.class, StringUtf8Coder.of(), inputConverter); - } - - /** - * Helper to read from Avro source given {@link AvroCoder}. Keep in mind that configuration - * object is altered to enable Avro input. - */ - public static HDFSFileSource, NullWritable> - fromAvro(String filepattern, final AvroCoder coder, Configuration conf) { - Class> formatClass = castClass(AvroKeyInputFormat.class); - SerializableFunction, NullWritable>, T> inputConverter = - new SerializableFunction, NullWritable>, T>() { - @Override - public T apply(KV, NullWritable> input) { - try { - return CoderUtils.clone(coder, input.getKey().datum()); - } catch (CoderException e) { - throw new RuntimeException(e); - } - } - }; - conf.set("avro.schema.input.key", coder.getSchema().toString()); - return from(filepattern, formatClass, coder, inputConverter).withConfiguration(conf); - } - - /** - * Helper to read from Avro source given {@link Schema}. Keep in mind that configuration - * object is altered to enable Avro input. - */ - public static HDFSFileSource, NullWritable> - fromAvro(String filepattern, Schema schema, Configuration conf) { - return fromAvro(filepattern, AvroCoder.of(schema), conf); - } - - /** - * Helper to read from Avro source given {@link Class}. Keep in mind that configuration - * object is altered to enable Avro input. - */ - public static HDFSFileSource, NullWritable> - fromAvro(String filepattern, Class cls, Configuration conf) { - return fromAvro(filepattern, AvroCoder.of(cls), conf); - } - - // ======================================================================= - // Builder methods - // ======================================================================= - - public abstract HDFSFileSource.Builder toBuilder(); - public static HDFSFileSource.Builder builder() { - return new AutoValue_HDFSFileSource.Builder<>(); - } - - /** - * AutoValue builder for {@link HDFSFileSource}. - */ - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setFilepattern(String filepattern); - public abstract Builder setFormatClass( - Class> formatClass); - public abstract Builder setCoder(Coder coder); - public abstract Builder setInputConverter( - SerializableFunction, T> inputConverter); - public abstract Builder setSerializableConfiguration( - SerializableConfiguration serializableConfiguration); - public Builder setConfiguration(Configuration configuration) { - if (configuration == null) { - configuration = new Configuration(false); - } - return this.setSerializableConfiguration(new SerializableConfiguration(configuration)); - } - public abstract Builder setSerializableSplit(SerializableSplit serializableSplit); - public abstract Builder setUsername(@Nullable String username); - public abstract Builder setValidateSource(boolean validate); - public abstract HDFSFileSource build(); - } - - public HDFSFileSource withConfiguration(@Nullable Configuration configuration) { - return this.toBuilder().setConfiguration(configuration).build(); - } - - public HDFSFileSource withUsername(@Nullable String username) { - return this.toBuilder().setUsername(username).build(); - } - - // ======================================================================= - // BoundedSource - // ======================================================================= - - @Override - public List> split( - final long desiredBundleSizeBytes, - PipelineOptions options) throws Exception { - if (serializableSplit() == null) { - List inputSplits = UGIHelper.getBestUGI(username()).doAs( - new PrivilegedExceptionAction>() { - @Override - public List run() throws Exception { - return computeSplits(desiredBundleSizeBytes, serializableConfiguration()); - } - }); - return Lists.transform(inputSplits, - new Function>() { - @Override - public BoundedSource apply(@Nullable InputSplit inputSplit) { - SerializableSplit serializableSplit = new SerializableSplit(inputSplit); - return HDFSFileSource.this.toBuilder() - .setSerializableSplit(serializableSplit) - .build(); - } - }); - } else { - return ImmutableList.of(this); - } - } - - @Override - public long getEstimatedSizeBytes(PipelineOptions options) { - long size = 0; - - try { - // If this source represents a split from split, - // then return the size of the split, rather then the entire input - if (serializableSplit() != null) { - return serializableSplit().getSplit().getLength(); - } - - size += UGIHelper.getBestUGI(username()).doAs(new PrivilegedExceptionAction() { - @Override - public Long run() throws Exception { - long size = 0; - Job job = SerializableConfiguration.newJob(serializableConfiguration()); - for (FileStatus st : listStatus(createFormat(job), job)) { - size += st.getLen(); - } - return size; - } - }); - } catch (IOException e) { - LOG.warn( - "Will estimate size of input to be 0 bytes. Can't estimate size of the input due to:", e); - // ignore, and return 0 - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOG.warn( - "Will estimate size of input to be 0 bytes. Can't estimate size of the input due to:", e); - // ignore, and return 0 - } - return size; - } - - @Override - public BoundedReader createReader(PipelineOptions options) throws IOException { - this.validate(); - return new HDFSFileReader<>(this, filepattern(), formatClass(), serializableSplit()); - } - - @Override - public void validate() { - if (validateSource()) { - try { - UGIHelper.getBestUGI(username()).doAs(new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - final Path pathPattern = new Path(filepattern()); - FileSystem fs = FileSystem.get(pathPattern.toUri(), - SerializableConfiguration.newConfiguration(serializableConfiguration())); - FileStatus[] fileStatuses = fs.globStatus(pathPattern); - checkState( - fileStatuses != null && fileStatuses.length > 0, - "Unable to find any files matching %s", filepattern()); - return null; - } - }); - } catch (IOException | InterruptedException e) { - throw new RuntimeException(e); - } - } - } - - @Override - public Coder getDefaultOutputCoder() { - return coder(); - } - - // ======================================================================= - // Helpers - // ======================================================================= - - private List computeSplits(long desiredBundleSizeBytes, - SerializableConfiguration serializableConfiguration) - throws IOException, IllegalAccessException, InstantiationException { - Job job = SerializableConfiguration.newJob(serializableConfiguration); - FileInputFormat.setMinInputSplitSize(job, desiredBundleSizeBytes); - FileInputFormat.setMaxInputSplitSize(job, desiredBundleSizeBytes); - return createFormat(job).getSplits(job); - } - - private FileInputFormat createFormat(Job job) - throws IOException, IllegalAccessException, InstantiationException { - Path path = new Path(filepattern()); - FileInputFormat.addInputPath(job, path); - return formatClass().newInstance(); - } - - private List listStatus(FileInputFormat format, Job job) - throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { - // FileInputFormat#listStatus is protected, so call using reflection - Method listStatus = FileInputFormat.class.getDeclaredMethod("listStatus", JobContext.class); - listStatus.setAccessible(true); - @SuppressWarnings("unchecked") - List stat = (List) listStatus.invoke(format, job); - return stat; - } - - @SuppressWarnings("unchecked") - private static Coder getDefaultCoder(Class c) { - if (Writable.class.isAssignableFrom(c)) { - Class writableClass = (Class) c; - return (Coder) WritableCoder.of(writableClass); - } else if (Void.class.equals(c)) { - return (Coder) VoidCoder.of(); - } - // TODO: how to use registered coders here? - throw new IllegalStateException("Cannot find coder for " + c); - } - - @SuppressWarnings("unchecked") - private static Class castClass(Class aClass) { - return (Class) aClass; - } - - // ======================================================================= - // BoundedReader - // ======================================================================= - - private static class HDFSFileReader extends BoundedSource.BoundedReader { - - private final HDFSFileSource source; - private final String filepattern; - private final Class> formatClass; - private final Job job; - - private List splits; - private ListIterator splitsIterator; - - private Configuration conf; - private FileInputFormat format; - private TaskAttemptContext attemptContext; - private RecordReader currentReader; - private KV currentPair; - - HDFSFileReader( - HDFSFileSource source, - String filepattern, - Class> formatClass, - SerializableSplit serializableSplit) - throws IOException { - this.source = source; - this.filepattern = filepattern; - this.formatClass = formatClass; - this.job = SerializableConfiguration.newJob(source.serializableConfiguration()); - - if (serializableSplit != null) { - this.splits = ImmutableList.of(serializableSplit.getSplit()); - this.splitsIterator = splits.listIterator(); - } - } - - @Override - public boolean start() throws IOException { - Path path = new Path(filepattern); - FileInputFormat.addInputPath(job, path); - - conf = job.getConfiguration(); - try { - format = formatClass.newInstance(); - } catch (InstantiationException | IllegalAccessException e) { - throw new IOException("Cannot instantiate file input format " + formatClass, e); - } - attemptContext = new TaskAttemptContextImpl(conf, new TaskAttemptID()); - - if (splitsIterator == null) { - splits = format.getSplits(job); - splitsIterator = splits.listIterator(); - } - - return advance(); - } - - @Override - public boolean advance() throws IOException { - try { - if (currentReader != null && currentReader.nextKeyValue()) { - currentPair = nextPair(); - return true; - } else { - while (splitsIterator.hasNext()) { - // advance the reader and see if it has records - final InputSplit nextSplit = splitsIterator.next(); - @SuppressWarnings("unchecked") - RecordReader reader = - (RecordReader) format.createRecordReader(nextSplit, attemptContext); - if (currentReader != null) { - currentReader.close(); - } - currentReader = reader; - UGIHelper.getBestUGI(source.username()).doAs(new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - currentReader.initialize(nextSplit, attemptContext); - return null; - } - }); - if (currentReader.nextKeyValue()) { - currentPair = nextPair(); - return true; - } - currentReader.close(); - currentReader = null; - } - // either no next split or all readers were empty - currentPair = null; - return false; - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IOException(e); - } - } - - @Override - public T getCurrent() throws NoSuchElementException { - if (currentPair == null) { - throw new NoSuchElementException(); - } - return source.inputConverter().apply(currentPair); - } - - @Override - public void close() throws IOException { - if (currentReader != null) { - currentReader.close(); - currentReader = null; - } - currentPair = null; - } - - @Override - public BoundedSource getCurrentSource() { - return source; - } - - @SuppressWarnings("unchecked") - private KV nextPair() throws IOException, InterruptedException { - K key = currentReader.getCurrentKey(); - V value = currentReader.getCurrentValue(); - // clone Writable objects since they are reused between calls to RecordReader#nextKeyValue - if (key instanceof Writable) { - key = (K) WritableUtils.clone((Writable) key, conf); - } - if (value instanceof Writable) { - value = (V) WritableUtils.clone((Writable) value, conf); - } - return KV.of(key, value); - } - - // ======================================================================= - // Optional overrides - // ======================================================================= - - @Override - public Double getFractionConsumed() { - if (currentReader == null) { - return 0.0; - } - if (splits.isEmpty()) { - return 1.0; - } - int index = splitsIterator.previousIndex(); - int numReaders = splits.size(); - if (index == numReaders) { - return 1.0; - } - double before = 1.0 * index / numReaders; - double after = 1.0 * (index + 1) / numReaders; - Double fractionOfCurrentReader = getProgress(); - if (fractionOfCurrentReader == null) { - return before; - } - return before + fractionOfCurrentReader * (after - before); - } - - private Double getProgress() { - try { - return (double) currentReader.getProgress(); - } catch (IOException | InterruptedException e) { - return null; - } - } - - } - - // ======================================================================= - // SerializableSplit - // ======================================================================= - - /** - * A wrapper to allow Hadoop {@link org.apache.hadoop.mapreduce.InputSplit}s to be - * serialized using Java's standard serialization mechanisms. Note that the InputSplit - * has to be Writable (which most are). - */ - protected static class SerializableSplit implements Externalizable { - private static final long serialVersionUID = 0L; - - private InputSplit split; - - public SerializableSplit() { - } - - public SerializableSplit(InputSplit split) { - checkArgument(split instanceof Writable, "Split is not writable: %s", split); - this.split = split; - } - - public InputSplit getSplit() { - return split; - } - - @Override - public void writeExternal(ObjectOutput out) throws IOException { - out.writeUTF(split.getClass().getCanonicalName()); - ((Writable) split).write(out); - } - - @Override - public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - String className = in.readUTF(); - try { - split = (InputSplit) Class.forName(className).newInstance(); - ((Writable) split).readFields(in); - } catch (InstantiationException | IllegalAccessException e) { - throw new IOException(e); - } - } - } - -} diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Sink.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Sink.java deleted file mode 100644 index fe2db5f37507..000000000000 --- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Sink.java +++ /dev/null @@ -1,195 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.hdfs; - -import java.io.Serializable; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.display.HasDisplayData; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; - -/** - * This class is deprecated, and only exists for HDFSFileSink. - */ -@Deprecated -public abstract class Sink implements Serializable, HasDisplayData { - /** - * Ensures that the sink is valid and can be written to before the write operation begins. One - * should use {@link com.google.common.base.Preconditions} to implement this method. - */ - public abstract void validate(PipelineOptions options); - - /** - * Returns an instance of a {@link WriteOperation} that can write to this Sink. - */ - public abstract WriteOperation createWriteOperation(); - - /** - * {@inheritDoc} - * - *

    By default, does not register any display data. Implementors may override this method - * to provide their own display data. - */ - @Override - public void populateDisplayData(DisplayData.Builder builder) {} - - /** - * A {@link WriteOperation} defines the process of a parallel write of objects to a Sink. - * - *

    The {@code WriteOperation} defines how to perform initialization and finalization of a - * parallel write to a sink as well as how to create a {@link Sink.Writer} object that can write - * a bundle to the sink. - * - *

    Since operations in Beam may be run multiple times for redundancy or fault-tolerance, - * the initialization and finalization defined by a WriteOperation must be idempotent. - * - *

    {@code WriteOperation}s may be mutable; a {@code WriteOperation} is serialized after the - * call to {@code initialize} method and deserialized before calls to - * {@code createWriter} and {@code finalized}. However, it is not - * reserialized after {@code createWriter}, so {@code createWriter} should not mutate the - * state of the {@code WriteOperation}. - * - *

    See {@link Sink} for more detailed documentation about the process of writing to a Sink. - * - * @param The type of objects to write - * @param The result of a per-bundle write - */ - public abstract static class WriteOperation implements Serializable { - /** - * Performs initialization before writing to the sink. Called before writing begins. - */ - public abstract void initialize(PipelineOptions options) throws Exception; - - /** - * Indicates that the operation will be performing windowed writes. - */ - public abstract void setWindowedWrites(boolean windowedWrites); - - /** - * Given an Iterable of results from bundle writes, performs finalization after writing and - * closes the sink. Called after all bundle writes are complete. - * - *

    The results that are passed to finalize are those returned by bundles that completed - * successfully. Although bundles may have been run multiple times (for fault-tolerance), only - * one writer result will be passed to finalize for each bundle. An implementation of finalize - * should perform clean up of any failed and successfully retried bundles. Note that these - * failed bundles will not have their writer result passed to finalize, so finalize should be - * capable of locating any temporary/partial output written by failed bundles. - * - *

    A best practice is to make finalize atomic. If this is impossible given the semantics - * of the sink, finalize should be idempotent, as it may be called multiple times in the case of - * failure/retry or for redundancy. - * - *

    Note that the iteration order of the writer results is not guaranteed to be consistent if - * finalize is called multiple times. - * - * @param writerResults an Iterable of results from successful bundle writes. - */ - public abstract void finalize(Iterable writerResults, PipelineOptions options) - throws Exception; - - /** - * Creates a new {@link Sink.Writer} to write a bundle of the input to the sink. - * - *

    The bundle id that the writer will use to uniquely identify its output will be passed to - * {@link Writer#openWindowed} or {@link Writer#openUnwindowed}. - * - *

    Must not mutate the state of the WriteOperation. - */ - public abstract Writer createWriter(PipelineOptions options) throws Exception; - - /** - * Returns the Sink that this write operation writes to. - */ - public abstract Sink getSink(); - - /** - * Returns a coder for the writer result type. - */ - public abstract Coder getWriterResultCoder(); - } - - /** - * A Writer writes a bundle of elements from a PCollection to a sink. - * {@link Writer#openWindowed} or {@link Writer#openUnwindowed} is called before writing begins - * and {@link Writer#close} is called after all elements in the bundle have been written. - * {@link Writer#write} writes an element to the sink. - * - *

    Note that any access to static members or methods of a Writer must be thread-safe, as - * multiple instances of a Writer may be instantiated in different threads on the same worker. - * - *

    See {@link Sink} for more detailed documentation about the process of writing to a Sink. - * - * @param The type of object to write - * @param The writer results type (e.g., the bundle's output filename, as String) - */ - public abstract static class Writer { - /** - * Performs bundle initialization. For example, creates a temporary file for writing or - * initializes any state that will be used across calls to {@link Writer#write}. - * - *

    The unique id that is given to open should be used to ensure that the writer's output does - * not interfere with the output of other Writers, as a bundle may be executed many times for - * fault tolerance. See {@link Sink} for more information about bundle ids. - * - *

    The window and paneInfo arguments are populated when windowed writes are requested. - * shard and numbShards are populated for the case of static sharding. In cases where the - * runner is dynamically picking sharding, shard and numShards might both be set to -1. - */ - public abstract void openWindowed(String uId, - BoundedWindow window, - PaneInfo paneInfo, - int shard, - int numShards) throws Exception; - - /** - * Perform bundle initialization for the case where the file is written unwindowed. - */ - public abstract void openUnwindowed(String uId, - int shard, - int numShards) throws Exception; - - public abstract void cleanup() throws Exception; - - /** - * Called for each value in the bundle. - */ - public abstract void write(T value) throws Exception; - - /** - * Finishes writing the bundle. Closes any resources used for writing the bundle. - * - *

    Returns a writer result that will be used in the {@link Sink.WriteOperation}'s - * finalization. The result should contain some way to identify the output of this bundle (using - * the bundle id). {@link WriteOperation#finalize} will use the writer result to identify - * successful writes. See {@link Sink} for more information about bundle ids. - * - * @return the writer result - */ - public abstract WriteT close() throws Exception; - - /** - * Returns the write operation this writer belongs to. - */ - public abstract WriteOperation getWriteOperation(); - - - } -} diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/UGIHelper.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/UGIHelper.java deleted file mode 100644 index fd05a19bea54..000000000000 --- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/UGIHelper.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.hdfs; - -import java.io.IOException; -import javax.annotation.Nullable; -import org.apache.hadoop.security.UserGroupInformation; - -/** - * {@link UserGroupInformation} helper methods. - */ -public class UGIHelper { - - /** - * Find the most appropriate UserGroupInformation to use. - * @param username the user name, or NULL if none is specified. - * @return the most appropriate UserGroupInformation - */ - public static UserGroupInformation getBestUGI(@Nullable String username) throws IOException { - return UserGroupInformation.getBestUGI(null, username); - } - -} diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Write.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Write.java deleted file mode 100644 index ef6556eb481d..000000000000 --- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/Write.java +++ /dev/null @@ -1,588 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.hdfs; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; -import javax.annotation.Nullable; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.hdfs.Sink.WriteOperation; -import org.apache.beam.sdk.io.hdfs.Sink.Writer; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.ValueProvider; -import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; -import org.apache.beam.sdk.transforms.Create; -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.View; -import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; -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.PCollection.IsBounded; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PDone; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This class is deprecated, and only exists currently for HDFSFileSink. - */ -@Deprecated -public class Write extends PTransform, PDone> { - private static final Logger LOG = LoggerFactory.getLogger(Write.class); - - private static final int UNKNOWN_SHARDNUM = -1; - private static final int UNKNOWN_NUMSHARDS = -1; - - private final Sink sink; - // This allows the number of shards to be dynamically computed based on the input - // PCollection. - @Nullable - private final PTransform, PCollectionView> computeNumShards; - // We don't use a side input for static sharding, as we want this value to be updatable - // when a pipeline is updated. - @Nullable - private final ValueProvider numShardsProvider; - private boolean windowedWrites; - - /** - * Creates a {@link Write} transform that writes to the given {@link Sink}, letting the runner - * control how many different shards are produced. - */ - public static Write to(Sink sink) { - checkNotNull(sink, "sink"); - return new Write<>(sink, null /* runner-determined sharding */, null, false); - } - - private Write( - Sink sink, - @Nullable PTransform, PCollectionView> computeNumShards, - @Nullable ValueProvider numShardsProvider, - boolean windowedWrites) { - this.sink = sink; - this.computeNumShards = computeNumShards; - this.numShardsProvider = numShardsProvider; - this.windowedWrites = windowedWrites; - } - - @Override - public PDone expand(PCollection input) { - checkArgument(IsBounded.BOUNDED == input.isBounded() || windowedWrites, - "%s can only be applied to an unbounded PCollection if doing windowed writes", - Write.class.getSimpleName()); - return createWrite(input, sink.createWriteOperation()); - } - - @Override - public void validate(PipelineOptions options) { - sink.validate(options); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder - .add(DisplayData.item("sink", sink.getClass()).withLabel("Write Sink")) - .include("sink", sink); - if (getSharding() != null) { - builder.include("sharding", getSharding()); - } else if (getNumShards() != null) { - String numShards = getNumShards().isAccessible() - ? getNumShards().get().toString() : getNumShards().toString(); - builder.add(DisplayData.item("numShards", numShards) - .withLabel("Fixed Number of Shards")); - } - } - - /** - * Returns the {@link Sink} associated with this PTransform. - */ - public Sink getSink() { - return sink; - } - - /** - * 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 - * {@link #withSharding(PTransform)}), or runner-determined (by {@link - * #withRunnerDeterminedSharding()}. - */ - @Nullable - public PTransform, PCollectionView> getSharding() { - return computeNumShards; - } - - public ValueProvider getNumShards() { - return numShardsProvider; - } - - /** - * Returns a new {@link Write} that will write to the current {@link Sink} using the - * specified number of shards. - * - *

    This option should be used sparingly as it can hurt performance. See {@link Write} for - * more information. - * - *

    A value less than or equal to 0 will be equivalent to the default behavior of - * runner-determined sharding. - */ - public Write withNumShards(int numShards) { - if (numShards > 0) { - return withNumShards(StaticValueProvider.of(numShards)); - } - return withRunnerDeterminedSharding(); - } - - /** - * Returns a new {@link Write} that will write to the current {@link Sink} using the - * {@link ValueProvider} specified number of shards. - * - *

    This option should be used sparingly as it can hurt performance. See {@link Write} for - * more information. - */ - public Write withNumShards(ValueProvider numShardsProvider) { - return new Write<>(sink, null, numShardsProvider, windowedWrites); - } - - /** - * Returns a new {@link Write} that will write to the current {@link Sink} using the - * specified {@link PTransform} to compute the number of shards. - * - *

    This option should be used sparingly as it can hurt performance. See {@link Write} for - * more information. - */ - public Write withSharding(PTransform, PCollectionView> sharding) { - checkNotNull( - sharding, "Cannot provide null sharding. Use withRunnerDeterminedSharding() instead"); - return new Write<>(sink, sharding, null, windowedWrites); - } - - /** - * Returns a new {@link Write} that will write to the current {@link Sink} with - * runner-determined sharding. - */ - public Write withRunnerDeterminedSharding() { - return new Write<>(sink, null, null, windowedWrites); - } - - /** - * Returns a new {@link Write} that writes preserves windowing on it's input. - * - *

    If this option is not specified, windowing and triggering are replaced by - * {@link GlobalWindows} and {@link DefaultTrigger}. - * - *

    If there is no data for a window, no output shards will be generated for that window. - * If a window triggers multiple times, then more than a single output shard might be - * generated multiple times; it's up to the sink implementation to keep these output shards - * unique. - * - *

    This option can only be used if {@link #withNumShards(int)} is also set to a - * positive value. - */ - public Write withWindowedWrites() { - return new Write<>(sink, computeNumShards, numShardsProvider, true); - } - - /** - * Writes all the elements in a bundle using a {@link Writer} produced by the - * {@link WriteOperation} associated with the {@link Sink}. - */ - private class WriteBundles extends DoFn { - // Writer that will write the records in this bundle. Lazily - // initialized in processElement. - private Writer writer = null; - private BoundedWindow window; - private final PCollectionView> writeOperationView; - - WriteBundles(PCollectionView> writeOperationView) { - this.writeOperationView = writeOperationView; - } - - @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) throws Exception { - // Lazily initialize the Writer - if (writer == null) { - WriteOperation writeOperation = c.sideInput(writeOperationView); - LOG.info("Opening writer for write operation {}", writeOperation); - writer = writeOperation.createWriter(c.getPipelineOptions()); - - if (windowedWrites) { - writer.openWindowed(UUID.randomUUID().toString(), window, c.pane(), UNKNOWN_SHARDNUM, - UNKNOWN_NUMSHARDS); - } else { - writer.openUnwindowed(UUID.randomUUID().toString(), UNKNOWN_SHARDNUM, UNKNOWN_NUMSHARDS); - } - this.window = window; - LOG.debug("Done opening writer {} for operation {}", writer, writeOperationView); - } - try { - writer.write(c.element()); - } catch (Exception e) { - // Discard write result and close the write. - try { - writer.close(); - // The writer does not need to be reset, as this DoFn cannot be reused. - } catch (Exception closeException) { - if (closeException instanceof InterruptedException) { - // Do not silently ignore interrupted state. - Thread.currentThread().interrupt(); - } - // Do not mask the exception that caused the write to fail. - e.addSuppressed(closeException); - } - throw e; - } - } - - @FinishBundle - public void finishBundle(FinishBundleContext c) throws Exception { - if (writer != null) { - WriteT result = writer.close(); - c.output(result, window.maxTimestamp(), window); - // Reset state in case of reuse. - writer = null; - window = null; - } - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.delegate(Write.this); - } - } - - /** - * Like {@link WriteBundles}, but where the elements for each shard have been collected into - * a single iterable. - * - * @see WriteBundles - */ - private class WriteShardedBundles extends DoFn>, WriteT> { - private final PCollectionView> writeOperationView; - private final PCollectionView numShardsView; - - WriteShardedBundles(PCollectionView> writeOperationView, - PCollectionView numShardsView) { - this.writeOperationView = writeOperationView; - this.numShardsView = numShardsView; - } - - @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) throws Exception { - int numShards = numShardsView != null ? c.sideInput(numShardsView) : getNumShards().get(); - // In a sharded write, single input element represents one shard. We can open and close - // the writer in each call to processElement. - WriteOperation writeOperation = c.sideInput(writeOperationView); - LOG.info("Opening writer for write operation {}", writeOperation); - Writer writer = writeOperation.createWriter(c.getPipelineOptions()); - if (windowedWrites) { - writer.openWindowed(UUID.randomUUID().toString(), window, c.pane(), c.element().getKey(), - numShards); - } else { - writer.openUnwindowed(UUID.randomUUID().toString(), UNKNOWN_SHARDNUM, UNKNOWN_NUMSHARDS); - } - LOG.debug("Done opening writer {} for operation {}", writer, writeOperationView); - - try { - try { - for (T t : c.element().getValue()) { - writer.write(t); - } - } catch (Exception e) { - try { - writer.close(); - } catch (Exception closeException) { - if (closeException instanceof InterruptedException) { - // Do not silently ignore interrupted state. - Thread.currentThread().interrupt(); - } - // Do not mask the exception that caused the write to fail. - e.addSuppressed(closeException); - } - throw e; - } - - // Close the writer; if this throws let the error propagate. - WriteT result = writer.close(); - c.output(result); - } catch (Exception e) { - // If anything goes wrong, make sure to delete the temporary file. - writer.cleanup(); - throw e; - } - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.delegate(Write.this); - } - } - - private static class ApplyShardingKey extends DoFn> { - private final PCollectionView numShardsView; - private final ValueProvider numShardsProvider; - private int shardNumber; - - ApplyShardingKey(PCollectionView numShardsView, - ValueProvider numShardsProvider) { - this.numShardsView = numShardsView; - this.numShardsProvider = numShardsProvider; - shardNumber = UNKNOWN_SHARDNUM; - } - - @ProcessElement - public void processElement(ProcessContext context) { - int shardCount = 0; - if (numShardsView != null) { - shardCount = context.sideInput(numShardsView); - } else { - checkNotNull(numShardsProvider); - shardCount = numShardsProvider.get(); - } - checkArgument( - shardCount > 0, - "Must have a positive number of shards specified for non-runner-determined sharding." - + " Got %s", - shardCount); - if (shardNumber == UNKNOWN_SHARDNUM) { - // We want to desynchronize the first record sharding key for each instance of - // ApplyShardingKey, so records in a small PCollection will be statistically balanced. - shardNumber = ThreadLocalRandom.current().nextInt(shardCount); - } else { - shardNumber = (shardNumber + 1) % shardCount; - } - context.output(KV.of(shardNumber, context.element())); - } - } - - /** - * A write is performed as sequence of three {@link ParDo}'s. - * - *

    In the first, a do-once ParDo is applied to a singleton PCollection containing the Sink's - * {@link WriteOperation}. In this initialization ParDo, {@link WriteOperation#initialize} is - * called. The output of this ParDo is a singleton PCollection - * containing the WriteOperation. - * - *

    This singleton collection containing the WriteOperation is then used as a side input to a - * ParDo over the PCollection of elements to write. In this bundle-writing phase, - * {@link WriteOperation#createWriter} is called to obtain a {@link Writer}. - * {@link Writer#open} and {@link Writer#close} are called in {@link DoFn#startBundle} and - * {@link DoFn#finishBundle}, respectively, and {@link Writer#write} method is called for - * every element in the bundle. The output of this ParDo is a PCollection of - * writer result objects (see {@link Sink} for a description of writer results)-one for - * each bundle. - * - *

    The final do-once ParDo uses the singleton collection of the WriteOperation as input and - * the collection of writer results as a side-input. In this ParDo, - * {@link WriteOperation#finalize} is called to finalize the write. - * - *

    If the write of any element in the PCollection fails, {@link Writer#close} will be called - * before the exception that caused the write to fail is propagated and the write result will be - * discarded. - * - *

    Since the {@link WriteOperation} is serialized after the initialization ParDo and - * deserialized in the bundle-writing and finalization phases, any state change to the - * WriteOperation object that occurs during initialization is visible in the latter phases. - * However, the WriteOperation is not serialized after the bundle-writing phase. This is why - * implementations should guarantee that {@link WriteOperation#createWriter} does not mutate - * WriteOperation). - */ - private PDone createWrite( - PCollection input, WriteOperation writeOperation) { - Pipeline p = input.getPipeline(); - writeOperation.setWindowedWrites(windowedWrites); - - // A coder to use for the WriteOperation. - @SuppressWarnings("unchecked") - Coder> operationCoder = - (Coder>) SerializableCoder.of(writeOperation.getClass()); - - // A singleton collection of the WriteOperation, to be used as input to a ParDo to initialize - // the sink. - PCollection> operationCollection = - p.apply("CreateOperationCollection", Create.of(writeOperation).withCoder(operationCoder)); - - // Initialize the resource in a do-once ParDo on the WriteOperation. - operationCollection = operationCollection - .apply("Initialize", ParDo.of( - new DoFn, WriteOperation>() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - WriteOperation writeOperation = c.element(); - LOG.info("Initializing write operation {}", writeOperation); - writeOperation.initialize(c.getPipelineOptions()); - writeOperation.setWindowedWrites(windowedWrites); - LOG.debug("Done initializing write operation {}", writeOperation); - // The WriteOperation is also the output of this ParDo, so it can have mutable - // state. - c.output(writeOperation); - } - })) - .setCoder(operationCoder); - - // Create a view of the WriteOperation to be used as a sideInput to the parallel write phase. - final PCollectionView> writeOperationView = - operationCollection.apply(View.>asSingleton()); - - if (!windowedWrites) { - // Re-window the data into the global window and remove any existing triggers. - input = - input.apply( - Window.into(new GlobalWindows()) - .triggering(DefaultTrigger.of()) - .discardingFiredPanes()); - } - - - // Perform the per-bundle writes as a ParDo on the input PCollection (with the WriteOperation - // as a side input) and collect the results of the writes in a PCollection. - // There is a dependency between this ParDo and the first (the WriteOperation PCollection - // as a side input), so this will happen after the initial ParDo. - PCollection results; - final PCollectionView numShardsView; - if (computeNumShards == null && numShardsProvider == null) { - if (windowedWrites) { - throw new IllegalStateException("When doing windowed writes, numShards must be set" - + "explicitly to a positive value"); - } - numShardsView = null; - results = input - .apply("WriteBundles", - ParDo.of(new WriteBundles<>(writeOperationView)) - .withSideInputs(writeOperationView)); - } else { - if (computeNumShards != null) { - numShardsView = input.apply(computeNumShards); - results = input - .apply("ApplyShardLabel", ParDo.of( - new ApplyShardingKey(numShardsView, null)).withSideInputs(numShardsView)) - .apply("GroupIntoShards", GroupByKey.create()) - .apply("WriteShardedBundles", - ParDo.of(new WriteShardedBundles<>(writeOperationView, numShardsView)) - .withSideInputs(numShardsView, writeOperationView)); - } else { - numShardsView = null; - results = input - .apply("ApplyShardLabel", ParDo.of(new ApplyShardingKey(null, numShardsProvider))) - .apply("GroupIntoShards", GroupByKey.create()) - .apply("WriteShardedBundles", - ParDo.of(new WriteShardedBundles<>(writeOperationView, null)) - .withSideInputs(writeOperationView)); - } - } - results.setCoder(writeOperation.getWriterResultCoder()); - - if (windowedWrites) { - // When processing streaming windowed writes, results will arrive multiple times. This - // means we can't share the below implementation that turns the results into a side input, - // as new data arriving into a side input does not trigger the listening DoFn. Instead - // we aggregate the result set using a singleton GroupByKey, so the DoFn will be triggered - // whenever new data arrives. - PCollection> keyedResults = - results.apply("AttachSingletonKey", WithKeys.of((Void) null)); - keyedResults.setCoder(KvCoder.of(VoidCoder.of(), writeOperation - .getWriterResultCoder())); - - // Is the continuation trigger sufficient? - keyedResults - .apply("FinalizeGroupByKey", GroupByKey.create()) - .apply("Finalize", ParDo.of(new DoFn>, Integer>() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - WriteOperation writeOperation = c.sideInput(writeOperationView); - LOG.info("Finalizing write operation {}.", writeOperation); - List results = Lists.newArrayList(c.element().getValue()); - writeOperation.finalize(results, c.getPipelineOptions()); - LOG.debug("Done finalizing write operation {}", writeOperation); - } - }).withSideInputs(writeOperationView)); - } else { - final PCollectionView> resultsView = - results.apply(View.asIterable()); - ImmutableList.Builder> sideInputs = - ImmutableList.>builder().add(resultsView); - if (numShardsView != null) { - sideInputs.add(numShardsView); - } - - // Finalize the write in another do-once ParDo on the singleton collection containing the - // Writer. The results from the per-bundle writes are given as an Iterable side input. - // The WriteOperation's state is the same as after its initialization in the first do-once - // ParDo. There is a dependency between this ParDo and the parallel write (the writer - // results collection as a side input), so it will happen after the parallel write. - // For the non-windowed case, we guarantee that if no data is written but the user has - // set numShards, then all shards will be written out as empty files. For this reason we - // use a side input here. - operationCollection - .apply("Finalize", ParDo.of(new DoFn, Integer>() { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - WriteOperation writeOperation = c.element(); - LOG.info("Finalizing write operation {}.", writeOperation); - List results = Lists.newArrayList(c.sideInput(resultsView)); - LOG.debug("Side input initialized to finalize write operation {}.", writeOperation); - - // We must always output at least 1 shard, and honor user-specified numShards if - // set. - int minShardsNeeded; - if (numShardsView != null) { - minShardsNeeded = c.sideInput(numShardsView); - } else if (numShardsProvider != null) { - minShardsNeeded = numShardsProvider.get(); - } else { - minShardsNeeded = 1; - } - int extraShardsNeeded = minShardsNeeded - results.size(); - if (extraShardsNeeded > 0) { - LOG.info( - "Creating {} empty output shards in addition to {} written for a total of " - + " {}.", extraShardsNeeded, results.size(), minShardsNeeded); - for (int i = 0; i < extraShardsNeeded; ++i) { - Writer writer = writeOperation.createWriter(c.getPipelineOptions()); - writer.openUnwindowed(UUID.randomUUID().toString(), UNKNOWN_SHARDNUM, - UNKNOWN_NUMSHARDS); - WriteT emptyWrite = writer.close(); - results.add(emptyWrite); - } - LOG.debug("Done creating extra shards."); - } - writeOperation.finalize(results, c.getPipelineOptions()); - LOG.debug("Done finalizing write operation {}", writeOperation); - } - }).withSideInputs(sideInputs.build())); - } - return PDone.in(input.getPipeline()); - } -} diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/package-info.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/package-info.java index 763b30a7b5ec..32c36cc89a7c 100644 --- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/package-info.java +++ b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/package-info.java @@ -17,6 +17,7 @@ */ /** - * Transforms used to read from the Hadoop file system (HDFS). + * {@link org.apache.beam.sdk.io.FileSystem} implementation for any Hadoop + * {@link org.apache.hadoop.fs.FileSystem}. */ package org.apache.beam.sdk.io.hdfs; diff --git a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSinkTest.java b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSinkTest.java deleted file mode 100644 index 9fa6606f04f6..000000000000 --- a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSinkTest.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.hdfs; - -import static org.junit.Assert.assertEquals; - -import com.google.common.base.MoreObjects; -import java.io.File; -import java.nio.charset.Charset; -import java.nio.file.Files; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.UUID; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.file.FileReader; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.mapred.AvroKey; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.values.KV; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -/** - * Tests for HDFSFileSinkTest. - */ -public class HDFSFileSinkTest { - - @Rule - public TemporaryFolder tmpFolder = new TemporaryFolder(); - - private final String part0 = "part-r-00000"; - private final String foobar = "foobar"; - - private void doWrite(Sink sink, - PipelineOptions options, - Iterable toWrite) throws Exception { - Sink.WriteOperation writeOperation = - (Sink.WriteOperation) sink.createWriteOperation(); - Sink.Writer writer = writeOperation.createWriter(options); - writer.openUnwindowed(UUID.randomUUID().toString(), -1, -1); - for (T t: toWrite) { - writer.write(t); - } - String writeResult = writer.close(); - writeOperation.finalize(Collections.singletonList(writeResult), options); - } - - @Test - public void testWriteSingleRecord() throws Exception { - PipelineOptions options = PipelineOptionsFactory.create(); - File file = tmpFolder.newFolder(); - - HDFSFileSink sink = - HDFSFileSink.to( - file.toString(), - SequenceFileOutputFormat.class, - NullWritable.class, - Text.class, - new SerializableFunction>() { - @Override - public KV apply(String input) { - return KV.of(NullWritable.get(), new Text(input)); - } - }); - - doWrite(sink, options, Collections.singletonList(foobar)); - - SequenceFile.Reader.Option opts = - SequenceFile.Reader.file(new Path(file.toString(), part0)); - SequenceFile.Reader reader = new SequenceFile.Reader(new Configuration(), opts); - assertEquals(NullWritable.class.getName(), reader.getKeyClassName()); - assertEquals(Text.class.getName(), reader.getValueClassName()); - NullWritable k = NullWritable.get(); - Text v = new Text(); - assertEquals(true, reader.next(k, v)); - assertEquals(NullWritable.get(), k); - assertEquals(new Text(foobar), v); - } - - @Test - public void testToText() throws Exception { - PipelineOptions options = PipelineOptionsFactory.create(); - File file = tmpFolder.newFolder(); - - HDFSFileSink sink = HDFSFileSink.toText(file.toString()); - - doWrite(sink, options, Collections.singletonList(foobar)); - - List strings = Files.readAllLines(new File(file.toString(), part0).toPath(), - Charset.forName("UTF-8")); - assertEquals(Collections.singletonList(foobar), strings); - } - - @DefaultCoder(AvroCoder.class) - static class GenericClass { - int intField; - String stringField; - public GenericClass() {} - public GenericClass(int intValue, String stringValue) { - this.intField = intValue; - this.stringField = stringValue; - } - @Override - public String toString() { - return MoreObjects.toStringHelper(getClass()) - .add("intField", intField) - .add("stringField", stringField) - .toString(); - } - @Override - public int hashCode() { - return Objects.hash(intField, stringField); - } - @Override - public boolean equals(Object other) { - if (other == null || !(other instanceof GenericClass)) { - return false; - } - GenericClass o = (GenericClass) other; - return Objects.equals(intField, o.intField) && Objects.equals(stringField, o.stringField); - } - } - - @Test - public void testToAvro() throws Exception { - PipelineOptions options = PipelineOptionsFactory.create(); - File file = tmpFolder.newFolder(); - - HDFSFileSink, NullWritable> sink = HDFSFileSink.toAvro( - file.toString(), - AvroCoder.of(GenericClass.class), - new Configuration(false)); - - doWrite(sink, options, Collections.singletonList(new GenericClass(3, "foobar"))); - - GenericDatumReader datumReader = new GenericDatumReader(); - FileReader reader = - DataFileReader.openReader(new File(file.getAbsolutePath(), part0 + ".avro"), datumReader); - GenericData.Record next = reader.next(null); - assertEquals("foobar", next.get("stringField").toString()); - assertEquals(3, next.get("intField")); - } - -} diff --git a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java deleted file mode 100644 index a96423901475..000000000000 --- a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java +++ /dev/null @@ -1,231 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.hdfs; - -import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.io.Source; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.SourceTestUtils; -import org.apache.beam.sdk.values.KV; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.SequenceFile.Writer; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -/** - * Tests for HDFSFileSource. - */ -public class HDFSFileSourceTest { - - private Random random = new Random(0L); - - @Rule - public TemporaryFolder tmpFolder = new TemporaryFolder(); - - @Test - public void testFullyReadSingleFile() throws Exception { - PipelineOptions options = PipelineOptionsFactory.create(); - List> expectedResults = createRandomRecords(3, 10, 0); - File file = createFileWithData("tmp.seq", expectedResults); - - HDFSFileSource, IntWritable, Text> source = - HDFSFileSource.from( - file.toString(), SequenceFileInputFormat.class, IntWritable.class, Text.class); - - assertEquals(file.length(), source.getEstimatedSizeBytes(null)); - - assertThat(expectedResults, containsInAnyOrder(readFromSource(source, options).toArray())); - } - - @Test - public void testFullyReadSingleFileWithSpaces() throws Exception { - PipelineOptions options = PipelineOptionsFactory.create(); - List> expectedResults = createRandomRecords(3, 10, 0); - File file = createFileWithData("tmp data.seq", expectedResults); - - HDFSFileSource, IntWritable, Text> source = - HDFSFileSource.from( - file.toString(), SequenceFileInputFormat.class, IntWritable.class, Text.class); - - assertEquals(file.length(), source.getEstimatedSizeBytes(null)); - - assertThat(expectedResults, containsInAnyOrder(readFromSource(source, options).toArray())); - } - - @Test - public void testFullyReadFilePattern() throws IOException { - PipelineOptions options = PipelineOptionsFactory.create(); - List> data1 = createRandomRecords(3, 10, 0); - File file1 = createFileWithData("file1", data1); - - List> data2 = createRandomRecords(3, 10, 10); - createFileWithData("file2", data2); - - List> data3 = createRandomRecords(3, 10, 20); - createFileWithData("file3", data3); - - List> data4 = createRandomRecords(3, 10, 30); - createFileWithData("otherfile", data4); - - List> expectedResults = new ArrayList<>(); - expectedResults.addAll(data1); - expectedResults.addAll(data2); - expectedResults.addAll(data3); - - HDFSFileSource, IntWritable, Text> source = - HDFSFileSource.from( - new File(file1.getParent(), "file*").toString(), SequenceFileInputFormat.class, - IntWritable.class, Text.class); - - assertThat(expectedResults, containsInAnyOrder(readFromSource(source, options).toArray())); - } - - @Test - public void testCloseUnstartedFilePatternReader() throws IOException { - PipelineOptions options = PipelineOptionsFactory.create(); - List> data1 = createRandomRecords(3, 10, 0); - File file1 = createFileWithData("file1", data1); - - List> data2 = createRandomRecords(3, 10, 10); - createFileWithData("file2", data2); - - List> data3 = createRandomRecords(3, 10, 20); - createFileWithData("file3", data3); - - List> data4 = createRandomRecords(3, 10, 30); - createFileWithData("otherfile", data4); - - HDFSFileSource, IntWritable, Text> source = - HDFSFileSource.from( - new File(file1.getParent(), "file*").toString(), - SequenceFileInputFormat.class, IntWritable.class, Text.class); - Source.Reader> reader = source.createReader(options); - - // Closing an unstarted FilePatternReader should not throw an exception. - try { - reader.close(); - } catch (Exception e) { - fail("Closing an unstarted FilePatternReader should not throw an exception"); - } - } - - @Test - public void testSplits() throws Exception { - PipelineOptions options = PipelineOptionsFactory.create(); - - List> expectedResults = createRandomRecords(3, 10000, 0); - File file = createFileWithData("tmp.seq", expectedResults); - - HDFSFileSource, IntWritable, Text> source = - HDFSFileSource.from( - file.toString(), SequenceFileInputFormat.class, IntWritable.class, Text.class); - - // Assert that the source produces the expected records - assertEquals(expectedResults, readFromSource(source, options)); - - // Split with a small bundle size (has to be at least size of sync interval) - List>> splits = source - .split(SequenceFile.SYNC_INTERVAL, options); - assertTrue(splits.size() > 2); - SourceTestUtils.assertSourcesEqualReferenceSource(source, splits, options); - int nonEmptySplits = 0; - for (BoundedSource> subSource : splits) { - if (readFromSource(subSource, options).size() > 0) { - nonEmptySplits += 1; - } - } - assertTrue(nonEmptySplits > 2); - } - - @Test - public void testSplitEstimatedSize() throws Exception { - PipelineOptions options = PipelineOptionsFactory.create(); - - List> expectedResults = createRandomRecords(3, 10000, 0); - File file = createFileWithData("tmp.avro", expectedResults); - - HDFSFileSource, IntWritable, Text> source = - HDFSFileSource.from(file.toString(), SequenceFileInputFormat.class, - IntWritable.class, Text.class); - - long originalSize = source.getEstimatedSizeBytes(options); - long splitTotalSize = 0; - List>> splits = source.split( - SequenceFile.SYNC_INTERVAL, options - ); - for (BoundedSource> splitSource : splits) { - splitTotalSize += splitSource.getEstimatedSizeBytes(options); - } - // Assert that the estimated size of the whole is the sum of its parts - assertEquals(originalSize, splitTotalSize); - } - - private File createFileWithData(String filename, List> records) - throws IOException { - File tmpFile = tmpFolder.newFile(filename); - try (Writer writer = SequenceFile.createWriter(new Configuration(), - Writer.keyClass(IntWritable.class), Writer.valueClass(Text.class), - Writer.file(new Path(tmpFile.toURI())))) { - - for (KV record : records) { - writer.append(record.getKey(), record.getValue()); - } - } - return tmpFile; - } - - private List> createRandomRecords(int dataItemLength, - int numItems, int offset) { - List> records = new ArrayList<>(); - for (int i = 0; i < numItems; i++) { - IntWritable key = new IntWritable(i + offset); - Text value = new Text(createRandomString(dataItemLength)); - records.add(KV.of(key, value)); - } - return records; - } - - private String createRandomString(int length) { - char[] chars = "abcdefghijklmnopqrstuvwxyz".toCharArray(); - StringBuilder builder = new StringBuilder(); - for (int i = 0; i < length; i++) { - builder.append(chars[random.nextInt(chars.length)]); - } - return builder.toString(); - } - -} From c2f47515693ef8046b3a6bb8a892af7b5a8df59b Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 4 May 2017 13:46:24 -0700 Subject: [PATCH 070/387] Run Optimize Imports on the Repository --- .../test/java/org/apache/beam/examples/WindowedWordCountIT.java | 1 - .../examples/complete/game/utils/WriteWindowedToBigQuery.java | 1 - .../beam/runners/apex/translation/TransformTranslator.java | 1 - .../apache/beam/runners/dataflow/util/DataflowTemplateJob.java | 1 - .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 1 - 5 files changed, 5 deletions(-) diff --git a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java index b5eddb5d17e7..f7e35c023172 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java @@ -30,7 +30,6 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.ThreadLocalRandom; - import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.examples.common.WriteOneFilePerWindow.PerWindowFiles; import org.apache.beam.sdk.PipelineResult; diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java index deb9db2c0fd5..db3319c34985 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/utils/WriteWindowedToBigQuery.java @@ -19,7 +19,6 @@ import com.google.api.services.bigquery.model.TableRow; import java.util.Map; - import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TransformTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TransformTranslator.java index 49ff49b4dbf3..c924b2ee53ef 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TransformTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/TransformTranslator.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.apex.translation; - import java.io.Serializable; import org.apache.beam.sdk.transforms.PTransform; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java index 2937184090a8..b4644620d038 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java @@ -19,7 +19,6 @@ import com.google.common.annotations.VisibleForTesting; import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.DataflowPipelineJob; -import org.apache.beam.sdk.PipelineResult.State; import org.joda.time.Duration; /** 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 fbbf862d1e84..6e410c7b6c28 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 @@ -38,7 +38,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; - import java.io.IOException; import java.util.List; import java.util.Map; From 2ac2a34b90b8d03ef71b01a75c14a9b16fa482b9 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 4 May 2017 14:07:36 -0700 Subject: [PATCH 071/387] Move AppliedPTransform into the Runners package This is a read-only view of the TransformHierarchy.Node, and is intended for use by Runner authors. --- .../src/main/java/org/apache/beam/runners/apex/ApexRunner.java | 2 +- .../beam/runners/apex/translation/TranslationContext.java | 2 +- .../runners/core/construction/DeduplicatedFlattenFactory.java | 2 +- .../runners/core/construction/EmptyFlattenAsCreateFactory.java | 2 +- .../beam/runners/core/construction/PTransformMatchers.java | 2 +- .../beam/runners/core/construction/PTransformReplacements.java | 2 +- .../org/apache/beam/runners/core/construction/PTransforms.java | 2 +- .../apache/beam/runners/core/construction/PrimitiveCreate.java | 2 +- .../apache/beam/runners/core/construction/SdkComponents.java | 2 +- .../runners/core/construction/UnsupportedOverrideFactory.java | 2 +- .../core/construction/EmptyFlattenAsCreateFactoryTest.java | 2 +- .../beam/runners/core/construction/PTransformMatchersTest.java | 2 +- .../runners/core/construction/PTransformReplacementsTest.java | 2 +- .../apache/beam/runners/core/construction/PTransformsTest.java | 2 +- .../beam/runners/core/construction/SdkComponentsTest.java | 2 +- .../construction/SingleInputOutputOverrideFactoryTest.java | 2 +- .../beam/runners/direct/BoundedReadEvaluatorFactory.java | 2 +- .../java/org/apache/beam/runners/direct/CommittedResult.java | 2 +- .../org/apache/beam/runners/direct/CompletionCallback.java | 2 +- .../direct/DirectGBKIntoKeyedWorkItemsOverrideFactory.java | 2 +- .../main/java/org/apache/beam/runners/direct/DirectGraph.java | 2 +- .../org/apache/beam/runners/direct/DirectGraphVisitor.java | 2 +- .../beam/runners/direct/DirectGroupByKeyOverrideFactory.java | 2 +- .../org/apache/beam/runners/direct/EmptyInputProvider.java | 2 +- .../java/org/apache/beam/runners/direct/EvaluationContext.java | 2 +- .../beam/runners/direct/ExecutorServiceParallelExecutor.java | 2 +- .../apache/beam/runners/direct/FlattenEvaluatorFactory.java | 2 +- .../beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java | 2 +- .../beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java | 2 +- .../beam/runners/direct/ImmutabilityEnforcementFactory.java | 2 +- .../java/org/apache/beam/runners/direct/ModelEnforcement.java | 2 +- .../apache/beam/runners/direct/ModelEnforcementFactory.java | 2 +- .../java/org/apache/beam/runners/direct/ParDoEvaluator.java | 2 +- .../org/apache/beam/runners/direct/ParDoEvaluatorFactory.java | 2 +- .../apache/beam/runners/direct/ParDoMultiOverrideFactory.java | 2 +- .../beam/runners/direct/PassthroughTransformEvaluator.java | 2 +- .../java/org/apache/beam/runners/direct/PipelineExecutor.java | 2 +- .../java/org/apache/beam/runners/direct/RootInputProvider.java | 2 +- .../org/apache/beam/runners/direct/RootProviderRegistry.java | 2 +- .../direct/SplittableProcessElementsEvaluatorFactory.java | 2 +- .../beam/runners/direct/StatefulParDoEvaluatorFactory.java | 2 +- .../main/java/org/apache/beam/runners/direct/StepAndKey.java | 2 +- .../org/apache/beam/runners/direct/StepTransformResult.java | 2 +- .../apache/beam/runners/direct/TestStreamEvaluatorFactory.java | 2 +- .../apache/beam/runners/direct/TransformEvaluatorFactory.java | 2 +- .../apache/beam/runners/direct/TransformEvaluatorRegistry.java | 2 +- .../java/org/apache/beam/runners/direct/TransformExecutor.java | 2 +- .../java/org/apache/beam/runners/direct/TransformResult.java | 2 +- .../beam/runners/direct/UnboundedReadEvaluatorFactory.java | 2 +- .../org/apache/beam/runners/direct/ViewEvaluatorFactory.java | 2 +- .../org/apache/beam/runners/direct/ViewOverrideFactory.java | 2 +- .../apache/beam/runners/direct/WatermarkCallbackExecutor.java | 2 +- .../java/org/apache/beam/runners/direct/WatermarkManager.java | 2 +- .../org/apache/beam/runners/direct/WindowEvaluatorFactory.java | 2 +- .../apache/beam/runners/direct/WriteWithShardingFactory.java | 2 +- .../beam/runners/direct/BoundedReadEvaluatorFactoryTest.java | 2 +- .../org/apache/beam/runners/direct/CommittedResultTest.java | 2 +- .../org/apache/beam/runners/direct/DirectGraphVisitorTest.java | 2 +- .../test/java/org/apache/beam/runners/direct/DirectGraphs.java | 2 +- .../runners/direct/DirectGroupByKeyOverrideFactoryTest.java | 2 +- .../org/apache/beam/runners/direct/EvaluationContextTest.java | 2 +- .../beam/runners/direct/FlattenEvaluatorFactoryTest.java | 2 +- .../runners/direct/ImmutabilityEnforcementFactoryTest.java | 2 +- .../org/apache/beam/runners/direct/ParDoEvaluatorTest.java | 2 +- .../beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java | 2 +- .../apache/beam/runners/direct/StepTransformResultTest.java | 2 +- .../beam/runners/direct/TestStreamEvaluatorFactoryTest.java | 2 +- .../org/apache/beam/runners/direct/TransformExecutorTest.java | 2 +- .../beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java | 2 +- .../apache/beam/runners/direct/ViewEvaluatorFactoryTest.java | 2 +- .../apache/beam/runners/direct/ViewOverrideFactoryTest.java | 2 +- .../beam/runners/direct/WatermarkCallbackExecutorTest.java | 2 +- .../org/apache/beam/runners/direct/WatermarkManagerTest.java | 2 +- .../beam/runners/direct/WriteWithShardingFactoryTest.java | 2 +- .../beam/runners/flink/FlinkBatchTranslationContext.java | 2 +- .../beam/runners/flink/FlinkStreamingPipelineTranslator.java | 2 +- .../beam/runners/flink/FlinkStreamingTranslationContext.java | 2 +- .../beam/runners/dataflow/BatchStatefulParDoOverrides.java | 2 +- .../org/apache/beam/runners/dataflow/BatchViewOverrides.java | 2 +- .../org/apache/beam/runners/dataflow/DataflowPipelineJob.java | 2 +- .../beam/runners/dataflow/DataflowPipelineTranslator.java | 2 +- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 2 +- .../beam/runners/dataflow/PrimitiveParDoSingleFactory.java | 2 +- .../apache/beam/runners/dataflow/ReshuffleOverrideFactory.java | 2 +- .../apache/beam/runners/dataflow/StreamingViewOverrides.java | 2 +- .../org/apache/beam/runners/dataflow/TransformTranslator.java | 2 +- .../apache/beam/runners/dataflow/DataflowPipelineJobTest.java | 2 +- .../beam/runners/dataflow/PrimitiveParDoSingleFactoryTest.java | 2 +- .../main/java/org/apache/beam/runners/spark/SparkRunner.java | 2 +- .../java/org/apache/beam/runners/spark/TestSparkRunner.java | 2 +- .../beam/runners/spark/translation/EvaluationContext.java | 2 +- .../spark/translation/streaming/TrackStreamingSourcesTest.java | 2 +- sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java | 2 +- .../beam/sdk/{transforms => runners}/AppliedPTransform.java | 3 ++- .../java/org/apache/beam/sdk/runners/PTransformMatcher.java | 1 - .../org/apache/beam/sdk/runners/PTransformOverrideFactory.java | 1 - .../java/org/apache/beam/sdk/runners/TransformHierarchy.java | 1 - .../core/src/test/java/org/apache/beam/sdk/PipelineTest.java | 2 +- 98 files changed, 96 insertions(+), 98 deletions(-) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{transforms => runners}/AppliedPTransform.java (96%) 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 e1828c3217c6..ede41bc4c351 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 @@ -48,9 +48,9 @@ import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.GloballyAsSingletonView; import org.apache.beam.sdk.transforms.Create; 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 c78028ec5720..9c20449d07e0 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 @@ -35,8 +35,8 @@ import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; 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.AppliedPTransform; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java index 13e7593057d2..bd65843b1632 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java @@ -22,8 +22,8 @@ import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.Flatten.PCollections; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java index a6982d4801f2..f5be89b10ab0 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java @@ -22,8 +22,8 @@ import java.util.Map; import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.Flatten.PCollections; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java index b2bd7d93ce9a..bfe24a02ab63 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformMatchers.java @@ -23,8 +23,8 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.io.WriteFiles; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformMatcher; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.PTransform; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java index 72a3425bada8..706a9564640b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java @@ -22,7 +22,7 @@ import java.util.Map; import java.util.Set; -import org.apache.beam.sdk.transforms.AppliedPTransform; +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.PValue; 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 7ec0863860b6..6d2c6b6fe053 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 @@ -26,7 +26,7 @@ import java.util.Map; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; -import org.apache.beam.sdk.transforms.AppliedPTransform; +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.PValue; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java index a791a0e20a03..f43d23b88b8b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java @@ -19,8 +19,8 @@ package org.apache.beam.runners.core.construction; import java.util.Map; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Create.Values; import org.apache.beam.sdk.transforms.PTransform; 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 3e7807221e49..2de8237c8f36 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 @@ -31,7 +31,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.Components; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.NameUtils; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java index efafa33b3513..3efbd7c59b06 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java @@ -19,8 +19,8 @@ package org.apache.beam.runners.core.construction; import java.util.Map; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java index c388878bf97d..0345fef730e6 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java @@ -24,12 +24,12 @@ import java.util.Collections; import java.util.Map; import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory.PTransformReplacement; import org.apache.beam.sdk.runners.PTransformOverrideFactory.ReplacementOutput; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.Flatten.PCollections; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java index 6271234db021..9ae236be0737 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java @@ -36,6 +36,7 @@ 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.runners.PTransformMatcher; import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.state.StateSpecs; @@ -45,7 +46,6 @@ import org.apache.beam.sdk.state.TimerSpecs; import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java index c6c50f067d1e..1318feb17ea4 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java @@ -24,8 +24,8 @@ import com.google.common.collect.ImmutableMap; import java.util.Collections; 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.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; 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/PTransformsTest.java index 4ef70c08e56c..41255448dee7 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/PTransformsTest.java @@ -38,8 +38,8 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; 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 f605e015cc09..82840d670973 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 @@ -35,8 +35,8 @@ import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.common.runner.v1.RunnerApi.Components; 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.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java index acca5cd28b84..80bb0a7a6f2c 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java @@ -22,9 +22,9 @@ import java.io.Serializable; import java.util.Map; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory.ReplacementOutput; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.SimpleFunction; 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 26f985129f2b..99a0fca2e72d 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 @@ -37,7 +37,7 @@ 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.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java index 99abdd31a2df..8c45449491b5 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java @@ -21,7 +21,7 @@ import com.google.auto.value.AutoValue; import java.util.Set; import javax.annotation.Nullable; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.View.CreatePCollectionView; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java index 7b5ef4b530a1..0af22c83707e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CompletionCallback.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; /** * A callback for completing a bundle of input. 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 112024341aa9..64eecc852afd 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 @@ -21,7 +21,7 @@ import org.apache.beam.runners.core.SplittableParDo.GBKIntoKeyedWorkItems; import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; 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 e163d83f3200..83b214aca89a 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 @@ -23,7 +23,7 @@ import java.util.Map; import java.util.Set; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; 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 c3421363bbe8..1ee8ceb9a7e5 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 @@ -27,8 +27,8 @@ import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java index 4eb036300c4e..c2eb5e72b842 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactory.java @@ -19,8 +19,8 @@ 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.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; 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 396cdeed2460..c36879a5f7b6 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 @@ -19,7 +19,7 @@ import java.util.Collection; import java.util.Collections; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; 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 7b6461103477..362ff91b893c 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 @@ -40,7 +40,7 @@ import org.apache.beam.runners.direct.WatermarkManager.FiredTimers; import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index 4da62d583281..02fb11aea315 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java @@ -50,7 +50,7 @@ import org.apache.beam.runners.direct.WatermarkManager.FiredTimers; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult.State; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java index 341ea4b4d2bf..1b169b8f8510 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.direct; import com.google.common.collect.Iterables; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.Flatten.PCollections; import org.apache.beam.sdk.transforms.PTransform; 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 83548589c1eb..41c797f840b2 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 @@ -42,7 +42,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java index 1ea8e76051e9..08623efca2dd 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java @@ -35,7 +35,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java index 8880af9eaa87..4a67fb4a6bfb 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java @@ -21,7 +21,7 @@ import java.util.Map; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.util.MutationDetector; import org.apache.beam.sdk.util.MutationDetectors; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java index d2e942470c90..da5127fdc5a8 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.direct; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java index 30f1d20e9ece..2317fc03a030 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcementFactory.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; /** * Creates {@link ModelEnforcement} instances for an {@link AppliedPTransform} on an input 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 4cfd16fec901..28fc68d293cb 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 @@ -31,7 +31,7 @@ import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; 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 39595d86afa7..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 @@ -25,7 +25,7 @@ import java.util.List; import java.util.Map; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; 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 c72bf61e2c5f..89903da1ce00 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 @@ -29,8 +29,8 @@ 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.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java index c57932c80f00..317749373a9b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.util.WindowedValue; class PassthroughTransformEvaluator implements TransformEvaluator { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java index 07212c709637..d954fa206eb0 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java @@ -20,7 +20,7 @@ import java.util.Collection; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult.State; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.joda.time.Duration; 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 88e0769d58f4..ce6951805117 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 @@ -20,7 +20,7 @@ import java.util.Collection; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; +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; 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 b06a41c3cd32..4b0c06d3611f 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 @@ -23,7 +23,7 @@ import java.util.Collection; import java.util.Map; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.Flatten.PCollections; import org.apache.beam.sdk.transforms.PTransform; 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 44f2e853f97c..27972335d703 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 @@ -35,7 +35,7 @@ import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternalsFactory; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; 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 02472b3e69f6..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 @@ -39,8 +39,8 @@ import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.ParDoMultiOverrideFactory.StatefulParDo; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.state.StateSpec; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java index e18b2ac9b6aa..5440dde25e45 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java @@ -19,7 +19,7 @@ import com.google.common.base.MoreObjects; import java.util.Objects; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; /** * A (Step, Key) pair. This is useful as a map key or cache key for things that are available diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java index 7e5f82433136..c7f78472f627 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java @@ -26,7 +26,7 @@ import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.sdk.metrics.MetricUpdates; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.joda.time.Instant; 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 ea989d754668..8b21d5affd60 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 @@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; import org.apache.beam.runners.core.construction.ReplacementOutputs; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.testing.TestStream.ElementEvent; @@ -36,7 +37,6 @@ import org.apache.beam.sdk.testing.TestStream.EventType; import org.apache.beam.sdk.testing.TestStream.ProcessingTimeEvent; import org.apache.beam.sdk.testing.TestStream.WatermarkEvent; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java index c1873592d18f..72e126062fbe 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java @@ -20,7 +20,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; /** 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 a00253a51a44..d0e622d7f934 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 @@ -31,7 +31,7 @@ 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.sdk.transforms.AppliedPTransform; +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; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java index 26c4f5cf7bce..8e1515bb1901 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutor.java @@ -24,7 +24,7 @@ import org.apache.beam.sdk.metrics.MetricUpdates; import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.metrics.MetricsEnvironment; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.util.WindowedValue; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java index 0b0790ef0551..8d7aeda72fcf 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java @@ -22,7 +22,7 @@ import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.sdk.metrics.MetricUpdates; -import org.apache.beam.sdk.transforms.AppliedPTransform; +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.transforms.windowing.BoundedWindow; 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 922a681248cd..cba826ccc98c 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 @@ -35,7 +35,7 @@ import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java index f4648e9e5618..057f4a1836ca 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java @@ -23,7 +23,7 @@ import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.StepTransformResult.Builder; import org.apache.beam.runners.direct.ViewOverrideFactory.WriteView; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.View.CreatePCollectionView; import org.apache.beam.sdk.util.WindowedValue; 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 d4fd18fa1414..b3bbac827fd3 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 @@ -24,8 +24,8 @@ import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Values; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java index ece5a56732fd..bbe503463a53 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java @@ -24,7 +24,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; 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 21ba734c017c..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 @@ -55,8 +55,8 @@ import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.state.TimeDomain; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java index 30d507be2d02..f4228d98b341 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java @@ -20,7 +20,7 @@ import com.google.common.collect.Iterables; import java.util.Collection; import javax.annotation.Nullable; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java index 24462e59eee2..65a5a19382c2 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java @@ -27,8 +27,8 @@ import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.sdk.io.WriteFiles; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java index df7c18e41f89..6180d2994a1a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java @@ -50,9 +50,9 @@ import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java index 3a7fdf3a913b..cf19dc20e7e6 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java @@ -27,8 +27,8 @@ import java.util.EnumSet; import java.util.List; import org.apache.beam.runners.direct.CommittedResult.OutputType; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; 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 2afb6c39caec..7f46a0e4c9c3 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 @@ -29,8 +29,8 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; 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 73ada196104f..2f048fa7cab2 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 @@ -18,7 +18,7 @@ package org.apache.beam.runners.direct; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.values.PValue; /** Test utilities for the {@link DirectRunner}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactoryTest.java index 28fef4c7e5d5..60ced7ec5885 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectGroupByKeyOverrideFactoryTest.java @@ -23,9 +23,9 @@ 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.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory.PTransformReplacement; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.values.KV; 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 077bb6865cfc..72b1bbcba188 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 @@ -44,10 +44,10 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.WithKeys; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java index 7dc01e6332a2..615016c9d4af 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java @@ -25,8 +25,8 @@ import com.google.common.collect.Iterables; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java index 1cd578601623..c0919b9509fc 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java @@ -19,8 +19,8 @@ import java.io.Serializable; import java.util.Collections; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; 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 88fd5d2f159d..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 @@ -33,8 +33,8 @@ import org.apache.beam.runners.core.ReadyCheckingSideInputReader; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; 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 f1ba57cf8c49..eb54d5c818dc 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 @@ -45,11 +45,11 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +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.ValueState; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java index 45ae678694ad..1ea854c538a2 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java @@ -24,8 +24,8 @@ import static org.junit.Assert.assertThat; import org.apache.beam.runners.direct.CommittedResult.OutputType; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; import org.hamcrest.Matchers; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java index d1712e0a1630..30c2bde974dc 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java @@ -30,9 +30,9 @@ import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.TestClock; import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.TestStreamIndex; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java index 1f95327dd65c..dc0ef7ca7828 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorTest.java @@ -36,8 +36,8 @@ import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.runners.direct.CommittedResult.OutputType; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index 68129ad09a90..ceb078be3058 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -57,9 +57,9 @@ import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java index d8869b2ca25d..419698e29798 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java @@ -27,8 +27,8 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.Values; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java index eda00a785e0c..024e15c4c536 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewOverrideFactoryTest.java @@ -30,11 +30,11 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.runners.direct.ViewOverrideFactory.WriteView; import org.apache.beam.sdk.Pipeline.PipelineVisitor; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory.PTransformReplacement; import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java index f4a53da28368..b66734600ee9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java @@ -23,8 +23,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index e1e6ab59cdd4..9528ac93285e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -43,9 +43,9 @@ import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java index a2b0c5ce7733..5a2a328beab0 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java @@ -48,8 +48,8 @@ import org.apache.beam.sdk.io.fs.MatchResult.Metadata; 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.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnTester; 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 e96d1c5ec5e4..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 @@ -23,7 +23,7 @@ import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; 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 42d75cf9a261..35d1bcd19f68 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 @@ -28,10 +28,10 @@ import org.apache.beam.runners.core.construction.UnconsumedReads; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo.MultiOutput; 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 45ee14d25a0d..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 @@ -25,7 +25,7 @@ import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java index 13ae9ee03b8d..4d9a57fbf977 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java @@ -26,8 +26,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index debaf59ef93e..ef2bfed39e7c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -57,7 +57,7 @@ import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.GloballyAsSingletonView; import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index aef31554c7bf..23084ed50498 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -45,7 +45,7 @@ import org.apache.beam.runners.dataflow.util.MonitoringUtil; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.metrics.MetricResults; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; 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 92f9c9823410..840bda846ed6 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 @@ -75,8 +75,8 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; 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 57da61b905b7..4198baea7517 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 @@ -99,12 +99,12 @@ import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.runners.TransformHierarchy.Node; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.GroupedValues; import org.apache.beam.sdk.transforms.DoFn; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java index 2e50cb5f7b4b..8611d3cad945 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java @@ -23,8 +23,8 @@ import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.common.runner.v1.RunnerApi.DisplayData; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java index 5814efd0ce47..cd9378c60822 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java @@ -20,8 +20,8 @@ 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.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/StreamingViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/StreamingViewOverrides.java index eb385de7fa3f..6c385d74085b 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/StreamingViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/StreamingViewOverrides.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.ListCoder; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.PTransform; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java index 52b3a31b1f4c..23949bd23208 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java @@ -23,7 +23,7 @@ import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.runners.dataflow.util.OutputReference; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.AppliedPTransform; +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; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java index 237493a3247a..f868a174ba65 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java @@ -55,9 +55,9 @@ import org.apache.beam.sdk.PipelineResult.State; import org.apache.beam.sdk.extensions.gcp.auth.TestCredential; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.NoopPathValidator; import org.apache.beam.sdk.values.PInput; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactoryTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactoryTest.java index e320036d62dc..4390e7345818 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactoryTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactoryTest.java @@ -27,9 +27,9 @@ import java.util.List; import org.apache.beam.runners.dataflow.PrimitiveParDoSingleFactory.ParDoSingle; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory.PTransformReplacement; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; 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 1a0c04234da2..16d9fdcfbbe7 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 @@ -46,9 +46,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PipelineOptionsValidator; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PCollection; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index a6851c49ded5..ada7fb58fd7b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -42,10 +42,10 @@ import org.apache.beam.sdk.io.BoundedReadFromUnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PBegin; 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 5d77e9162e7d..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 @@ -31,7 +31,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; 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 3dcab26e946b..33a636ad65f8 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 @@ -33,8 +33,8 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.PTransform; 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 f7c3f24c46bd..1769c0522c9c 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 @@ -33,6 +33,7 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.runners.PTransformOverrideFactory.PTransformReplacement; @@ -40,7 +41,6 @@ import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.runners.TransformHierarchy.Node; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.UserCodeException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AppliedPTransform.java similarity index 96% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AppliedPTransform.java index 4e049a590d7e..7a3532e8117a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/AppliedPTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AppliedPTransform.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.transforms; +package org.apache.beam.sdk.runners; import com.google.auto.value.AutoValue; import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java index 6378ecc82f6e..350d0d98f7a6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformMatcher.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java index a28f303688d6..5f012dedca06 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; 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 703aeb684346..fac558bc8432 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 @@ -38,7 +38,6 @@ import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.runners.PTransformOverrideFactory.ReplacementOutput; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java index fda64b5a85e1..d6b527ccf6ee 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java @@ -40,6 +40,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformMatcher; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PTransformOverrideFactory; @@ -51,7 +52,6 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.ValidatesRunner; -import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.Flatten; From 72be5c71561bf552c25e2de2b0d21aa374b17ec0 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 5 May 2017 09:20:10 -0700 Subject: [PATCH 072/387] Convert Coder into an Abstract Static Class --- runners/google-cloud-dataflow-java/pom.xml | 2 +- .../org/apache/beam/sdk/coders/Coder.java | 36 +++++++++---------- .../beam/sdk/coders/CoderFactories.java | 35 +++++++++--------- .../beam/sdk/coders/StructuredCoder.java | 2 +- .../DoFnSignaturesSplittableDoFnTest.java | 4 +-- 5 files changed, 40 insertions(+), 39 deletions(-) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index e21b6de0f784..c7142fe024b2 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -33,7 +33,7 @@ jar - beam-master-20170504-2 + beam-master-20170505-wd-2914 1 6 diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index c923719eb025..169e448623c7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -57,10 +57,10 @@ * * @param the type of the values being transcoded */ -public interface Coder extends Serializable { +public abstract class Coder implements Serializable { /** The context in which encoding or decoding is being done. */ @Deprecated - class Context { + public static class Context { /** * The outer context: the value being encoded or decoded takes * up the remainder of the record/stream contents. @@ -118,7 +118,7 @@ public String toString() { * for some reason * @throws CoderException if the value could not be encoded for some reason */ - void encode(T value, OutputStream outStream) + public abstract void encode(T value, OutputStream outStream) throws CoderException, IOException; /** @@ -130,7 +130,7 @@ void encode(T value, OutputStream outStream) * @throws CoderException if the value could not be encoded for some reason */ @Deprecated - void encodeOuter(T value, OutputStream outStream) + public abstract void encodeOuter(T value, OutputStream outStream) throws CoderException, IOException; /** @@ -142,7 +142,7 @@ void encodeOuter(T value, OutputStream outStream) * @throws CoderException if the value could not be encoded for some reason */ @Deprecated - void encode(T value, OutputStream outStream, Context context) + public abstract void encode(T value, OutputStream outStream, Context context) throws CoderException, IOException; /** @@ -153,7 +153,7 @@ void encode(T value, OutputStream outStream, Context context) * for some reason * @throws CoderException if the value could not be decoded for some reason */ - T decode(InputStream inStream) throws CoderException, IOException; + public abstract T decode(InputStream inStream) throws CoderException, IOException; /** * Decodes a value of type {@code T} from the given input stream in @@ -164,7 +164,7 @@ void encode(T value, OutputStream outStream, Context context) * @throws CoderException if the value could not be decoded for some reason */ @Deprecated - T decodeOuter(InputStream inStream) throws CoderException, IOException; + public abstract T decodeOuter(InputStream inStream) throws CoderException, IOException; /** * Decodes a value of type {@code T} from the given input stream in @@ -175,7 +175,7 @@ void encode(T value, OutputStream outStream, Context context) * @throws CoderException if the value could not be decoded for some reason */ @Deprecated - T decode(InputStream inStream, Context context) + public abstract T decode(InputStream inStream, Context context) throws CoderException, IOException; /** @@ -184,7 +184,7 @@ T decode(InputStream inStream, Context context) * returns {@code null} if this cannot be done or this is not a * parameterized type. */ - List> getCoderArguments(); + public abstract List> getCoderArguments(); /** * Throw {@link NonDeterministicException} if the coding is not deterministic. @@ -202,7 +202,7 @@ T decode(InputStream inStream, Context context) * * @throws Coder.NonDeterministicException if this coder is not deterministic. */ - void verifyDeterministic() throws Coder.NonDeterministicException; + public abstract void verifyDeterministic() throws Coder.NonDeterministicException; /** * Returns {@code true} if this {@link Coder} is injective with respect to {@link Objects#equals}. @@ -214,7 +214,7 @@ T decode(InputStream inStream, Context context) * whenever {@code equals()} compares object identity, rather than performing a * semantic/structural comparison. */ - boolean consistentWithEquals(); + public abstract boolean consistentWithEquals(); /** * Returns an object with an {@code Object.equals()} method that represents structural equality @@ -234,7 +234,7 @@ T decode(InputStream inStream, Context context) * *

    See also {@link #consistentWithEquals()}. */ - Object structuralValue(T value); + public abstract Object structuralValue(T value); /** * Returns whether {@link #registerByteSizeObserver} cheap enough to @@ -246,7 +246,7 @@ T decode(InputStream inStream, Context context) * {@link org.apache.beam.sdk.runners.PipelineRunner} * implementations. */ - boolean isRegisterByteSizeObserverCheap(T value); + public abstract boolean isRegisterByteSizeObserverCheap(T value); /** * Returns whether {@link #registerByteSizeObserver} cheap enough to @@ -259,7 +259,7 @@ T decode(InputStream inStream, Context context) * implementations. */ @Deprecated - boolean isRegisterByteSizeObserverCheap(T value, Context context); + public abstract boolean isRegisterByteSizeObserverCheap(T value, Context context); /** * Notifies the {@code ElementByteSizeObserver} about the byte size @@ -269,7 +269,7 @@ T decode(InputStream inStream, Context context) * {@link org.apache.beam.sdk.runners.PipelineRunner} * implementations. */ - void registerByteSizeObserver( + public abstract void registerByteSizeObserver( T value, ElementByteSizeObserver observer) throws Exception; @@ -282,7 +282,7 @@ void registerByteSizeObserver( * implementations. */ @Deprecated - void registerByteSizeObserver( + public abstract void registerByteSizeObserver( T value, ElementByteSizeObserver observer, Context context) throws Exception; @@ -290,13 +290,13 @@ void registerByteSizeObserver( * Returns the {@link TypeDescriptor} for the type encoded. */ @Experimental(Kind.CODER_TYPE_ENCODING) - TypeDescriptor getEncodedTypeDescriptor(); + public abstract TypeDescriptor getEncodedTypeDescriptor(); /** * Exception thrown by {@link Coder#verifyDeterministic()} if the encoding is * not deterministic, including details of why the encoding is not deterministic. */ - class NonDeterministicException extends Exception { + public static class NonDeterministicException extends Exception { private Coder coder; private List reasons; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java index 0031698c0353..2a1d792b7603 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.coders; +import static com.google.common.base.Preconditions.checkArgument; + import com.google.common.base.MoreObjects; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -70,7 +72,12 @@ private CoderFactories() { } // Static utility class * will produce a {@code Coder>} for any {@code Coder Coder}. */ public static CoderFactory fromStaticMethods(Class clazz) { - return new CoderFactoryFromStaticMethods(clazz); + checkArgument( + Coder.class.isAssignableFrom(clazz), + "%s is not a subtype of %s", + clazz.getName(), + Coder.class.getSimpleName()); + return new CoderFactoryFromStaticMethods((Class) clazz); } /** @@ -142,7 +149,7 @@ public List getInstanceComponents(Object value) { * Returns a CoderFactory that invokes the given static factory method * to create the Coder. */ - private CoderFactoryFromStaticMethods(Class coderClazz) { + private CoderFactoryFromStaticMethods(Class coderClazz) { this.factoryMethod = getFactoryMethod(coderClazz); this.getComponentsMethod = getInstanceComponentsMethod(coderClazz); } @@ -203,8 +210,8 @@ private Method getFactoryMethod(Class coderClazz) { * each corresponding to an argument of the {@code of} * method. */ - private Method getInstanceComponentsMethod(Class coderClazz) { - TypeDescriptor coderType = TypeDescriptor.of(coderClazz); + private Method getInstanceComponentsMethod(Class coderClazz) { + TypeDescriptor coderType = TypeDescriptor.of(coderClazz); TypeDescriptor argumentType = getCodedType(coderType); // getInstanceComponents may be implemented in a superclass, @@ -235,19 +242,13 @@ private Method getInstanceComponentsMethod(Class coderClazz) { * If {@code coderType} is a subclass of {@link Coder} for a specific * type {@code T}, returns {@code T.class}. Otherwise, raises IllegalArgumentException. */ - private TypeDescriptor getCodedType(TypeDescriptor coderType) { - for (TypeDescriptor ifaceType : coderType.getInterfaces()) { - if (ifaceType.getRawType().equals(Coder.class)) { - ParameterizedType coderIface = (ParameterizedType) ifaceType.getType(); - @SuppressWarnings("unchecked") - TypeDescriptor token = - (TypeDescriptor) TypeDescriptor.of(coderIface.getActualTypeArguments()[0]); - return token; - } - } - throw new IllegalArgumentException( - "cannot build CoderFactory from class " + coderType - + ": does not implement Coder for any T."); + private TypeDescriptor getCodedType(TypeDescriptor coderType) { + TypeDescriptor coderSupertype = coderType.getSupertype(Coder.class); + ParameterizedType coderIface = (ParameterizedType) coderSupertype.getType(); + @SuppressWarnings("unchecked") + TypeDescriptor token = + (TypeDescriptor) TypeDescriptor.of(coderIface.getActualTypeArguments()[0]); + return token; } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java index cc39429a8e20..0c72618c2108 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java @@ -43,7 +43,7 @@ * expensive. * */ -public abstract class StructuredCoder implements Coder { +public abstract class StructuredCoder extends Coder { protected StructuredCoder() {} /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java index b937e84c5c96..07b3348fe101 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java @@ -25,8 +25,8 @@ import com.google.common.base.Predicates; import com.google.common.collect.Iterables; import java.util.List; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.BoundedPerElement; import org.apache.beam.sdk.transforms.DoFn.UnboundedPerElement; @@ -57,7 +57,7 @@ private static class SomeRestriction {} private abstract static class SomeRestrictionTracker implements RestrictionTracker {} - private abstract static class SomeRestrictionCoder implements Coder {} + private abstract static class SomeRestrictionCoder extends StructuredCoder {} @Test public void testHasRestrictionTracker() throws Exception { From 60f86db6ef211aedd7c7343842b68390ccf52d93 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 5 May 2017 10:20:38 -0700 Subject: [PATCH 073/387] [BEAM-59] Fully delete IOChannelUtils --- .../apache/beam/sdk/util/IOChannelUtils.java | 27 ------------------- 1 file changed, 27 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java deleted file mode 100644 index b658983eea87..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.util; - -import org.apache.beam.sdk.options.PipelineOptions; - -/** Do not use, being removed. */ -@Deprecated -public class IOChannelUtils { - @Deprecated - public static void registerIOFactoriesAllowOverride(PipelineOptions options) {} -} From 9740c6ee2dca01c8de03383fbc62485ff6dd982d Mon Sep 17 00:00:00 2001 From: Daniel Halperin Date: Fri, 5 May 2017 12:10:16 -0700 Subject: [PATCH 074/387] [BEAM-2141] disable beam_PerformanceTests_Dataflow Has not passed in Jenkins memory, at least multiple weeks. --- .test-infra/jenkins/job_beam_PerformanceTests_Dataflow.groovy | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.test-infra/jenkins/job_beam_PerformanceTests_Dataflow.groovy b/.test-infra/jenkins/job_beam_PerformanceTests_Dataflow.groovy index 51c73f34d59e..2c8f7ef182bf 100644 --- a/.test-infra/jenkins/job_beam_PerformanceTests_Dataflow.groovy +++ b/.test-infra/jenkins/job_beam_PerformanceTests_Dataflow.groovy @@ -40,4 +40,7 @@ job('beam_PerformanceTests_Dataflow'){ ] common_job_properties.buildPerformanceTest(delegate, argMap) + + // [BEAM-2141] Perf tests do not pass. + disabled() } From c640e7437871313303636def48d63acd678a3430 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 5 May 2017 10:36:40 -0700 Subject: [PATCH 075/387] Move PipelineRunner to toplevel sdk package (automated refactor) This allows excluding the runner-author-only sdk.runners package from the public javadoc. --- .../java/org/apache/beam/runners/apex/ApexRunner.java | 2 +- .../apache/beam/runners/apex/ApexRunnerRegistrar.java | 2 +- .../org/apache/beam/runners/apex/TestApexRunner.java | 2 +- .../main/java/org/apache/beam/runners/core/OldDoFn.java | 4 ++-- .../org/apache/beam/runners/direct/DirectRegistrar.java | 2 +- .../org/apache/beam/runners/direct/DirectRunner.java | 2 +- .../apache/beam/runners/direct/DoFnLifecycleManager.java | 2 +- .../org/apache/beam/runners/direct/DirectRunnerTest.java | 2 +- .../java/org/apache/beam/runners/flink/FlinkRunner.java | 2 +- .../apache/beam/runners/flink/FlinkRunnerRegistrar.java | 2 +- .../org/apache/beam/runners/flink/TestFlinkRunner.java | 2 +- .../beam/runners/dataflow/DataflowPipelineRegistrar.java | 2 +- .../org/apache/beam/runners/dataflow/DataflowRunner.java | 2 +- .../runners/dataflow/testing/TestDataflowRunner.java | 2 +- .../java/org/apache/beam/runners/spark/SparkRunner.java | 2 +- .../apache/beam/runners/spark/SparkRunnerDebugger.java | 2 +- .../apache/beam/runners/spark/SparkRunnerRegistrar.java | 2 +- .../org/apache/beam/runners/spark/TestSparkRunner.java | 2 +- .../core/src/main/java/org/apache/beam/sdk/Pipeline.java | 1 - .../apache/beam/sdk/{runners => }/PipelineRunner.java | 4 +--- .../src/main/java/org/apache/beam/sdk/coders/Coder.java | 9 +++++---- .../java/org/apache/beam/sdk/coders/package-info.java | 2 +- .../src/main/java/org/apache/beam/sdk/io/AvroSource.java | 2 +- .../org/apache/beam/sdk/options/PipelineOptions.java | 2 +- .../apache/beam/sdk/options/PipelineOptionsFactory.java | 2 +- .../src/main/java/org/apache/beam/sdk/package-info.java | 2 +- .../beam/sdk/runners/PTransformOverrideFactory.java | 1 + .../apache/beam/sdk/runners/PipelineRunnerRegistrar.java | 1 + .../java/org/apache/beam/sdk/state/StateContext.java | 3 ++- .../java/org/apache/beam/sdk/testing/CrashingRunner.java | 2 +- .../java/org/apache/beam/sdk/testing/NeedsRunner.java | 2 +- .../main/java/org/apache/beam/sdk/testing/PAssert.java | 2 +- .../java/org/apache/beam/sdk/testing/TestStream.java | 2 +- .../apache/beam/sdk/transforms/CombineWithContext.java | 3 ++- .../main/java/org/apache/beam/sdk/transforms/DoFn.java | 6 +++--- .../org/apache/beam/sdk/transforms/Materialization.java | 2 +- .../main/java/org/apache/beam/sdk/transforms/ParDo.java | 2 +- .../main/java/org/apache/beam/sdk/transforms/View.java | 2 +- .../java/org/apache/beam/sdk/values/PCollectionView.java | 3 ++- .../src/test/java/org/apache/beam/sdk/PipelineTest.java | 1 - .../beam/sdk/options/PipelineOptionsFactoryTest.java | 2 +- .../org/apache/beam/sdk/runners/PipelineRunnerTest.java | 1 + .../org/apache/beam/sdk/testing/CrashingRunnerTest.java | 2 +- .../sdk/transforms/display/DisplayDataEvaluator.java | 2 +- .../org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- .../org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java | 2 +- .../apache/beam/sdk/io/gcp/datastore/DatastoreV1.java | 3 ++- .../java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java | 2 +- .../src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java | 4 ++-- 49 files changed, 59 insertions(+), 55 deletions(-) rename sdks/java/core/src/main/java/org/apache/beam/sdk/{runners => }/PipelineRunner.java (94%) 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 ede41bc4c351..a50e10ef53d8 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 @@ -43,6 +43,7 @@ import org.apache.beam.runners.core.construction.PrimitiveCreate; import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.ListCoder; @@ -50,7 +51,6 @@ import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverride; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.GloballyAsSingletonView; import org.apache.beam.sdk.transforms.Create; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java index 8cde692f8c15..2aa34ef15789 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerRegistrar.java @@ -20,9 +20,9 @@ import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableList; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; /** diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java index a64ac549ec70..e068db086442 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java @@ -19,9 +19,9 @@ import java.io.IOException; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; -import org.apache.beam.sdk.runners.PipelineRunner; import org.joda.time.Duration; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java index 419c83779057..41bb598795e6 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OldDoFn.java @@ -18,9 +18,9 @@ package org.apache.beam.runners.core; import java.io.Serializable; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.ParDo; @@ -71,7 +71,7 @@ public abstract class Context { /** * Returns the {@code PipelineOptions} specified with the - * {@link org.apache.beam.sdk.runners.PipelineRunner} + * {@link PipelineRunner} * invoking this {@code OldDoFn}. The {@code PipelineOptions} will * be the default running via {@link DoFnTester}. */ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java index 9a34d3d9dfb4..3e69e2b527d2 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java @@ -19,9 +19,9 @@ import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableList; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; /** 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 984598a6faaf..b0ce5eb02845 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 @@ -34,12 +34,12 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.metrics.MetricResults; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.PTransformOverride; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java index cd644a66585b..0e30e5c07bb7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java @@ -26,7 +26,7 @@ import java.util.Collection; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.Setup; import org.apache.beam.sdk.transforms.DoFn.Teardown; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java index 83881fc46f24..428c6fc43d71 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java @@ -43,6 +43,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineResult.State; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; @@ -55,7 +56,6 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; 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 a5972efa1a15..80ef7bb32acd 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 @@ -31,10 +31,10 @@ import java.util.TreeSet; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.View; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java index 681459a819d7..ebc6adccf76c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java @@ -20,9 +20,9 @@ import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableList; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java index 8f50105a55b9..01b67e517d8e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java @@ -20,10 +20,10 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PipelineOptionsValidator; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.util.UserCodeException; /** diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java index 5bd3bcd6eb9a..b6802bb4b0aa 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java @@ -20,9 +20,9 @@ import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableList; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; /** 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 4198baea7517..5278a4a576d0 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 @@ -77,6 +77,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.PipelineResult.State; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; @@ -102,7 +103,6 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.Combine; diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java index c238d80bc933..ce9191586d1f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/testing/TestDataflowRunner.java @@ -38,8 +38,8 @@ import org.apache.beam.runners.dataflow.util.MonitoringUtil.JobMessagesHandler; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult.State; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; 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 16d9fdcfbbe7..8c02f0f0e90d 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 @@ -41,13 +41,13 @@ import org.apache.beam.runners.spark.translation.streaming.SparkRunnerStreamingContextFactory; import org.apache.beam.runners.spark.util.GlobalWatermarkHolder.WatermarksListener; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.PTransform; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java index 11c52c7a2ff3..6c74146bd61e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java @@ -25,9 +25,9 @@ import org.apache.beam.runners.spark.translation.TransformTranslator; import org.apache.beam.runners.spark.translation.streaming.StreamingTransformTranslator; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.joda.time.Duration; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java index bf926dc54ecc..e2e5ceba0221 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java @@ -20,9 +20,9 @@ import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableList; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index ada7fb58fd7b..6d10b7588a42 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -39,13 +39,13 @@ import org.apache.beam.runners.spark.util.GlobalWatermarkHolder; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.io.BoundedReadFromUnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PBegin; 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 1769c0522c9c..f4da6ad9c1c2 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 @@ -38,7 +38,6 @@ import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.runners.PTransformOverrideFactory.PTransformReplacement; import org.apache.beam.sdk.runners.PTransformOverrideFactory.ReplacementOutput; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.Create; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineRunner.java similarity index 94% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineRunner.java index 229e04ffa12d..18e79eb7fa27 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineRunner.java @@ -15,12 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.runners; +package org.apache.beam.sdk; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index 169e448623c7..061e9e535338 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -29,6 +29,7 @@ import java.util.Arrays; import java.util.List; import javax.annotation.Nullable; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; @@ -243,7 +244,7 @@ public abstract T decode(InputStream inStream, Context context) * constant time (or lazily). * *

    Not intended to be called by user code, but instead by - * {@link org.apache.beam.sdk.runners.PipelineRunner} + * {@link PipelineRunner} * implementations. */ public abstract boolean isRegisterByteSizeObserverCheap(T value); @@ -255,7 +256,7 @@ public abstract T decode(InputStream inStream, Context context) * constant time (or lazily). * *

    Not intended to be called by user code, but instead by - * {@link org.apache.beam.sdk.runners.PipelineRunner} + * {@link PipelineRunner} * implementations. */ @Deprecated @@ -266,7 +267,7 @@ public abstract T decode(InputStream inStream, Context context) * of the encoded value using this {@code Coder}. * *

    Not intended to be called by user code, but instead by - * {@link org.apache.beam.sdk.runners.PipelineRunner} + * {@link PipelineRunner} * implementations. */ public abstract void registerByteSizeObserver( @@ -278,7 +279,7 @@ public abstract void registerByteSizeObserver( * of the encoded value using this {@code Coder}. * *

    Not intended to be called by user code, but instead by - * {@link org.apache.beam.sdk.runners.PipelineRunner} + * {@link PipelineRunner} * implementations. */ @Deprecated diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java index 84e59e018b45..5693077759c4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/package-info.java @@ -27,7 +27,7 @@ * machines. * *

    Exactly when PCollection elements are encoded during execution depends on which - * {@link org.apache.beam.sdk.runners.PipelineRunner} is being used and how that runner + * {@link org.apache.beam.sdk.PipelineRunner} is being used and how that runner * chooses to execute the pipeline. As such, Beam requires that all PCollections have an * appropriate Coder in case it becomes necessary. In many cases, the Coder can be inferred from * the available Java type diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java index 96d21c674658..61bc4a4f1563 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java @@ -48,12 +48,12 @@ import org.apache.avro.io.DecoderFactory; import org.apache.avro.reflect.ReflectData; import org.apache.avro.reflect.ReflectDatumReader; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.values.PCollection; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; import org.apache.commons.compress.compressors.snappy.SnappyCompressorInputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java index 37081fde1a79..9a4d25ae3797 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java @@ -30,9 +30,9 @@ import java.util.concurrent.atomic.AtomicLong; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.ProxyInvocationHandler.Deserializer; import org.apache.beam.sdk.options.ProxyInvocationHandler.Serializer; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.joda.time.DateTimeUtils; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java index 0f6bedfc8b9d..c0990cb108a8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java @@ -73,8 +73,8 @@ import java.util.TreeMap; import java.util.TreeSet; import javax.annotation.Nonnull; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.Validation.Required; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.StringUtils; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java index 166c27651dc2..995bcb900854 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/package-info.java @@ -27,7 +27,7 @@ * the PTransforms consume and produce. * *

    Each Pipeline has a - * {@link org.apache.beam.sdk.runners.PipelineRunner} to specify + * {@link org.apache.beam.sdk.PipelineRunner} to specify * where and how it should run after pipeline construction is complete. * */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java index 5f012dedca06..58314c40058c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java @@ -21,6 +21,7 @@ import com.google.auto.value.AutoValue; import java.util.Map; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.annotations.Internal; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java index 41fd6f0029ba..c6303dded4ec 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java @@ -19,6 +19,7 @@ import com.google.auto.service.AutoService; import java.util.ServiceLoader; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Internal; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java index 52177cc0a41c..b7dd8e2681cf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateContext.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.state; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -30,7 +31,7 @@ public interface StateContext { /** * Returns the {@code PipelineOptions} specified with the - * {@link org.apache.beam.sdk.runners.PipelineRunner}. + * {@link PipelineRunner}. */ PipelineOptions getPipelineOptions(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java index b3360ae6583e..0dea27460470 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CrashingRunner.java @@ -20,8 +20,8 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.PipelineRunner; /** * A {@link PipelineRunner} that applies no overrides and throws an exception on calls to diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/NeedsRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/NeedsRunner.java index 5f22bee631e8..8ebc76c66d58 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/NeedsRunner.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/NeedsRunner.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.PipelineRunner; /** * Category tag for validation tests which utilize {@link TestPipeline} for execution and expect diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java index b5d7db5fc204..6e2b8c66fa41 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java @@ -34,6 +34,7 @@ import java.util.NoSuchElementException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.IterableCoder; @@ -42,7 +43,6 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java index 446c8a5d1974..d41b9ef5dc23 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java @@ -25,8 +25,8 @@ import com.google.common.collect.ImmutableList; import java.util.List; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PBegin; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java index 9ae19f8bfdd1..a7601b3cfc2c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.values.PCollectionView; @@ -36,7 +37,7 @@ public class CombineWithContext { public abstract static class Context { /** * Returns the {@code PipelineOptions} specified with the - * {@link org.apache.beam.sdk.runners.PipelineRunner} + * {@link PipelineRunner} * invoking this {@code KeyedCombineFn}. */ public abstract PipelineOptions getPipelineOptions(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index befba1dd71d7..e711ac2f297a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -88,7 +88,7 @@ public abstract class DoFn implements Serializable, HasDisplayD public abstract class StartBundleContext { /** * Returns the {@code PipelineOptions} specified with the {@link - * org.apache.beam.sdk.runners.PipelineRunner} invoking this {@code DoFn}. The {@code + * org.apache.beam.sdk.PipelineRunner} invoking this {@code DoFn}. The {@code * PipelineOptions} will be the default running via {@link DoFnTester}. */ public abstract PipelineOptions getPipelineOptions(); @@ -100,7 +100,7 @@ public abstract class StartBundleContext { public abstract class FinishBundleContext { /** * Returns the {@code PipelineOptions} specified with the {@link - * org.apache.beam.sdk.runners.PipelineRunner} invoking this {@code DoFn}. The {@code + * org.apache.beam.sdk.PipelineRunner} invoking this {@code DoFn}. The {@code * PipelineOptions} will be the default running via {@link DoFnTester}. */ public abstract PipelineOptions getPipelineOptions(); @@ -136,7 +136,7 @@ public abstract void output( public abstract class WindowedContext { /** * Returns the {@code PipelineOptions} specified with the {@link - * org.apache.beam.sdk.runners.PipelineRunner} invoking this {@code DoFn}. The {@code + * org.apache.beam.sdk.PipelineRunner} invoking this {@code DoFn}. The {@code * PipelineOptions} will be the default running via {@link DoFnTester}. */ public abstract PipelineOptions getPipelineOptions(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java index 6fb8c297f547..0d02b32d3cc3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Materialization.java @@ -18,8 +18,8 @@ package org.apache.beam.sdk.transforms; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.runners.PipelineRunner; /** * For internal use only; no backwards-compatibility guarantees. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index c45311a7b0ab..e67dbe153a5e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -29,10 +29,10 @@ import java.util.List; import java.util.Map; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.transforms.DoFn.WindowedContext; import org.apache.beam.sdk.transforms.display.DisplayData; 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 d7b81455b63f..bcbdb24d5290 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 @@ -19,10 +19,10 @@ import java.util.List; import java.util.Map; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.KV; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java index f89041a6124e..7d874124a9bd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java @@ -19,6 +19,7 @@ import java.io.Serializable; import javax.annotation.Nullable; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.annotations.Internal; @@ -36,7 +37,7 @@ * *

    A {@link PCollectionView} should always be the output of a * {@link org.apache.beam.sdk.transforms.PTransform}. It is the joint responsibility of - * this transform and each {@link org.apache.beam.sdk.runners.PipelineRunner} to implement + * this transform and each {@link PipelineRunner} to implement * the view in a runner-specific manner. * *

    The most common case is using the {@link View} transforms to prepare a {@link PCollection} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java index d6b527ccf6ee..2cc3f04203d1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java @@ -44,7 +44,6 @@ import org.apache.beam.sdk.runners.PTransformMatcher; import org.apache.beam.sdk.runners.PTransformOverride; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.testing.CrashingRunner; import org.apache.beam.sdk.testing.ExpectedLogs; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java index 76a5f18c0397..d40b5fcc0dc6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java @@ -53,7 +53,7 @@ import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; import org.apache.beam.sdk.testing.CrashingRunner; import org.apache.beam.sdk.testing.ExpectedLogs; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java index 76d8627b19ed..c5d7fbf7cf23 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertTrue; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.CrashingRunner; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java index c66aa50cb57f..62c5134ce67a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java @@ -22,9 +22,9 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Create; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java index 8f57f45b0ac4..6e8ca5e5db2f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java @@ -21,10 +21,10 @@ import java.util.Objects; import java.util.Set; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; 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 6e410c7b6c28..304864a7ca85 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 @@ -43,6 +43,7 @@ import java.util.Map; import java.util.regex.Pattern; import javax.annotation.Nullable; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -66,7 +67,6 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; 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 0e97c1254d93..846d07bb78a2 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 @@ -42,6 +42,7 @@ import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentLinkedQueue; import javax.annotation.Nullable; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; @@ -52,7 +53,6 @@ import org.apache.beam.sdk.io.range.ByteKeyRange; import org.apache.beam.sdk.io.range.ByteKeyRangeTracker; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; 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 fd4fccf0ba1d..d6464dd95855 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 @@ -69,6 +69,7 @@ import java.util.List; import java.util.NoSuchElementException; import javax.annotation.Nullable; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.SerializableCoder; @@ -192,7 +193,7 @@ * by providing the host port information through {@code withLocalhost("host:port"} for all the * above transforms. In such a case, all the Cloud Datastore API calls are directed to the Emulator. * - * @see org.apache.beam.sdk.runners.PipelineRunner + * @see PipelineRunner */ public class DatastoreV1 { 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 51da111ccd24..ac6cb44be291 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 @@ -31,6 +31,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import javax.annotation.Nullable; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -44,7 +45,6 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java index a34fb0f3cda0..7255a94357eb 100644 --- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java +++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java @@ -26,6 +26,7 @@ import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; import javax.xml.bind.ValidationEventHandler; +import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.CompressedSource; import org.apache.beam.sdk.io.FileBasedSink; @@ -34,7 +35,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PBegin; @@ -102,7 +102,7 @@ public class XmlIO { * *

    Permissions

    * - *

    Permission requirements depend on the {@link org.apache.beam.sdk.runners.PipelineRunner + *

    Permission requirements depend on the {@link PipelineRunner * PipelineRunner} that is used to execute the Beam pipeline. Please refer to the documentation of * corresponding {@link PipelineRunner PipelineRunners} for more details. * From b64f9acb116e92fb9ad35304a770aae92338a404 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 5 May 2017 10:33:11 -0700 Subject: [PATCH 076/387] Exclude sdk.runners from javadoc --- sdks/java/javadoc/ant.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/java/javadoc/ant.xml b/sdks/java/javadoc/ant.xml index 288f48bf9baa..9a1880acaa17 100644 --- a/sdks/java/javadoc/ant.xml +++ b/sdks/java/javadoc/ant.xml @@ -90,6 +90,7 @@ + From 15be5df5d83dd6d151e4b2ace0df8286a378b008 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 5 May 2017 10:39:26 -0700 Subject: [PATCH 077/387] Javadoc that the sdk.runners package is internal --- .../main/java/org/apache/beam/sdk/runners/package-info.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java index bb6cce6079cb..272693643d60 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/package-info.java @@ -16,8 +16,8 @@ * limitations under the License. */ /** - * Defines different data processing backends (aka, runners) for executing Beam Pipelines. + * For internal use only; no backwards compatibility guarantees. - *

    The runner is specified as part of the {@link org.apache.beam.sdk.options.PipelineOptions}. + *

    Internals for use by runners. */ package org.apache.beam.sdk.runners; From c6fe1f92fd0a49c58649ffb1dd40f53f14e098b9 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Fri, 5 May 2017 11:43:20 -0700 Subject: [PATCH 078/387] Use uuid for temp directory --- sdks/python/apache_beam/io/fileio.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/io/fileio.py b/sdks/python/apache_beam/io/fileio.py index 49562f796d59..bfed96ea59e2 100644 --- a/sdks/python/apache_beam/io/fileio.py +++ b/sdks/python/apache_beam/io/fileio.py @@ -23,6 +23,7 @@ import os import re import time +import uuid from apache_beam.internal import util from apache_beam.io import iobase @@ -155,8 +156,7 @@ def _create_temp_dir(self, file_path_prefix): 'at least two components.', file_path_prefix) path_components = [base_path, - 'beam-temp-' + last_component + time.strftime( - '-%Y-%m-%d_%H-%M-%S')] + 'beam-temp-' + last_component + '-' + uuid.uuid1().hex] return FileSystems.join(*path_components) @check_accessible(['file_path_prefix', 'file_name_suffix']) From 655947b597972d9fd6e1d3a777970b0b1152fa05 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 5 May 2017 10:08:32 -0700 Subject: [PATCH 079/387] Re-add AtomicCoder This is a moderately useful base class for coders which take no configuration. --- .../apache/beam/sdk/coders/AtomicCoder.java | 86 +++++++++++++++++++ 1 file changed, 86 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java new file mode 100644 index 000000000000..528cfb06d47b --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.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.coders; + +import java.util.Collections; +import java.util.List; + +/** + * A {@link Coder} that has no component {@link Coder Coders} or other state. + * + *

    Note that, unless the behavior is overridden, atomic coders are presumed to be deterministic. + * + *

    All atomic coders of the same class are considered to be equal to each other. As a result, + * an {@link AtomicCoder} should have no associated state. + * + * @param the type of the values being transcoded + */ +public abstract class AtomicCoder extends StructuredCoder { + /** + * Returns an empty list. + * + *

    {@link CoderFactories#fromStaticMethods(Class)} builds a {@link CoderFactory} from the + * {@code #of()} method and this method, used to determine the components of an object. Because + * {@link AtomicCoder} has no components, always returns an empty list. + * + * @param exampleValue unused, but part of the latent interface expected by {@link + * CoderFactories#fromStaticMethods} + */ + public static List getInstanceComponents(T exampleValue) { + return Collections.emptyList(); + } + + /** + * {@inheritDoc}. + * + * @throws NonDeterministicException + */ + @Override + public void verifyDeterministic() throws NonDeterministicException {} + + @Override + public List> getCoderArguments() { + return null; + } + + /** + * {@inheritDoc}. + * + * @return the empty {@link List}. + */ + @Override + public final List> getComponents() { + return Collections.emptyList(); + } + + /** + * {@inheritDoc}. + * + * @return true if the other object has the same class as this {@link AtomicCoder}. + */ + @Override + public final boolean equals(Object other) { + return other != null && this.getClass().equals(other.getClass()); + } + + @Override + public final int hashCode() { + return this.getClass().hashCode(); + } +} From 63258c6986866b5bef58043b056d5c0dfec7303f Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 5 May 2017 10:10:40 -0700 Subject: [PATCH 080/387] Add default implementations of Coder methods to Coder Remove from StructuredCoder. These are sensible defaults implemented in terms of other Coder methods. --- .../org/apache/beam/sdk/coders/Coder.java | 121 +++++++++++++++--- .../beam/sdk/coders/StructuredCoder.java | 67 +--------- 2 files changed, 108 insertions(+), 80 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index 061e9e535338..41e83acf6e50 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -22,6 +22,9 @@ import com.google.common.base.Joiner; import com.google.common.base.MoreObjects; import com.google.common.base.Objects; +import com.google.common.io.ByteStreams; +import com.google.common.io.CountingOutputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -205,6 +208,30 @@ public abstract T decode(InputStream inStream, Context context) */ public abstract void verifyDeterministic() throws Coder.NonDeterministicException; + /** + * Verifies all of the provided coders are deterministic. If any are not, throws a {@link + * NonDeterministicException} for the {@code target} {@link Coder}. + */ + public static void verifyDeterministic(Coder target, String message, Iterable> coders) + throws NonDeterministicException { + for (Coder coder : coders) { + try { + coder.verifyDeterministic(); + } catch (NonDeterministicException e) { + throw new NonDeterministicException(target, message, e); + } + } + } + + /** + * Verifies all of the provided coders are deterministic. If any are not, throws a {@link + * NonDeterministicException} for the {@code target} {@link Coder}. + */ + public static void verifyDeterministic(Coder target, String message, Coder... coders) + throws NonDeterministicException { + verifyDeterministic(target, message, Arrays.asList(coders)); + } + /** * Returns {@code true} if this {@link Coder} is injective with respect to {@link Objects#equals}. * @@ -214,28 +241,50 @@ public abstract T decode(InputStream inStream, Context context) *

    This condition is most notably false for arrays. More generally, this condition is false * whenever {@code equals()} compares object identity, rather than performing a * semantic/structural comparison. + * + *

    By default, returns false. */ - public abstract boolean consistentWithEquals(); + public boolean consistentWithEquals() { + return false; + } /** - * Returns an object with an {@code Object.equals()} method that represents structural equality - * on the argument. + * Returns an object with an {@code Object.equals()} method that represents structural equality on + * the argument. * *

    For any two values {@code x} and {@code y} of type {@code T}, if their encoded bytes are the * same, then it must be the case that {@code structuralValue(x).equals(@code structuralValue(y)}. * *

    Most notably: + * *

      *
    • The structural value for an array coder should perform a structural comparison of the - * contents of the arrays, rather than the default behavior of comparing according to object - * identity. - *
    • The structural value for a coder accepting {@code null} should be a proper object with - * an {@code equals()} method, even if the input value is {@code null}. + * contents of the arrays, rather than the default behavior of comparing according to object + * identity. + *
    • The structural value for a coder accepting {@code null} should be a proper object with an + * {@code equals()} method, even if the input value is {@code null}. *
    * *

    See also {@link #consistentWithEquals()}. + * + *

    By default, if this coder is {@link #consistentWithEquals()}, and the value is not null, + * returns the provided object. Otherwise, encodes the value into a {@code byte[]}, and returns + * an object that performs array equality on the encoded bytes. */ - public abstract Object structuralValue(T value); + public Object structuralValue(T value) { + if (value != null && consistentWithEquals()) { + return value; + } else { + try { + ByteArrayOutputStream os = new ByteArrayOutputStream(); + encode(value, os, Context.OUTER); + return new StructuralByteArray(os.toByteArray()); + } catch (Exception exn) { + throw new IllegalArgumentException( + "Unable to encode element '" + value + "' with coder '" + this + "'.", exn); + } + } + } /** * Returns whether {@link #registerByteSizeObserver} cheap enough to @@ -246,21 +295,44 @@ public abstract T decode(InputStream inStream, Context context) *

    Not intended to be called by user code, but instead by * {@link PipelineRunner} * implementations. + * + *

    By default, returns false. The default {@link #registerByteSizeObserver} implementation + * invokes {@link #getEncodedElementByteSize} which requires re-encoding an element + * unless it is overridden. This is considered expensive. */ - public abstract boolean isRegisterByteSizeObserverCheap(T value); + public boolean isRegisterByteSizeObserverCheap(T value) { + return isRegisterByteSizeObserverCheap(value, Context.NESTED); + } /** - * Returns whether {@link #registerByteSizeObserver} cheap enough to - * call for every element, that is, if this {@code Coder} can - * calculate the byte size of the element to be coded in roughly - * constant time (or lazily). + * {@inheritDoc} * *

    Not intended to be called by user code, but instead by * {@link PipelineRunner} * implementations. + * + * @return {@code false} unless it is overridden. {@link StructuredCoder#registerByteSizeObserver} + * invokes {@link #getEncodedElementByteSize} which requires re-encoding an element + * unless it is overridden. This is considered expensive. */ @Deprecated - public abstract boolean isRegisterByteSizeObserverCheap(T value, Context context); + public boolean isRegisterByteSizeObserverCheap(T value, Context context) { + return false; + } + + /** + * Returns the size in bytes of the encoded value using this coder. + */ + protected long getEncodedElementByteSize(T value, Context context) + throws Exception { + try (CountingOutputStream os = new CountingOutputStream(ByteStreams.nullOutputStream())) { + encode(value, os, context); + return os.getCount(); + } catch (Exception exn) { + throw new IllegalArgumentException( + "Unable to encode element '" + value + "' with coder '" + this + "'.", exn); + } + } /** * Notifies the {@code ElementByteSizeObserver} about the byte size @@ -269,10 +341,14 @@ public abstract T decode(InputStream inStream, Context context) *

    Not intended to be called by user code, but instead by * {@link PipelineRunner} * implementations. + * + *

    By default, this notifies {@code observer} about the byte size + * of the encoded value using this coder as returned by {@link #getEncodedElementByteSize}. */ - public abstract void registerByteSizeObserver( - T value, ElementByteSizeObserver observer) - throws Exception; + public void registerByteSizeObserver(T value, ElementByteSizeObserver observer) + throws Exception { + registerByteSizeObserver(value, observer, Context.NESTED); + } /** * Notifies the {@code ElementByteSizeObserver} about the byte size @@ -283,15 +359,20 @@ public abstract void registerByteSizeObserver( * implementations. */ @Deprecated - public abstract void registerByteSizeObserver( + public void registerByteSizeObserver( T value, ElementByteSizeObserver observer, Context context) - throws Exception; + throws Exception { + observer.update(getEncodedElementByteSize(value, context)); + } /** * Returns the {@link TypeDescriptor} for the type encoded. */ @Experimental(Kind.CODER_TYPE_ENCODING) - public abstract TypeDescriptor getEncodedTypeDescriptor(); + public TypeDescriptor getEncodedTypeDescriptor(){ + return (TypeDescriptor) + TypeDescriptor.of(getClass()).resolveType(new TypeDescriptor() {}.getType()); + } /** * Exception thrown by {@link Coder#verifyDeterministic()} if the encoding is diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java index 0c72618c2108..437f10d0b1c0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuredCoder.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.coders; -import com.google.common.io.ByteStreams; -import com.google.common.io.CountingOutputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; @@ -26,13 +24,15 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.TypeDescriptor; /** * An abstract base class to implement a {@link Coder} that defines equality, hashing, and printing * via the class name and recursively using {@link #getComponents}. * + *

    A {@link StructuredCoder} should be defined purely in terms of its component coders, and + * contain no additional configuration. + * *

    To extend {@link StructuredCoder}, override the following methods as appropriate: * *

      @@ -101,12 +101,14 @@ public String toString() { return builder.toString(); } + @Override public void encode(T value, OutputStream outStream) throws CoderException, IOException { encode(value, outStream, Coder.Context.NESTED); } @Deprecated + @Override public void encodeOuter(T value, OutputStream outStream) throws CoderException, IOException { encode(value, outStream, Coder.Context.OUTER); @@ -122,11 +124,13 @@ public void encode(T value, OutputStream outStream, Coder.Context context) } } + @Override public T decode(InputStream inStream) throws CoderException, IOException { return decode(inStream, Coder.Context.NESTED); } @Deprecated + @Override public T decodeOuter(InputStream inStream) throws CoderException, IOException { return decode(inStream, Coder.Context.OUTER); } @@ -141,63 +145,6 @@ public T decode(InputStream inStream, Coder.Context context) } } - /** - * {@inheritDoc} - * - * @return {@code false} unless it is overridden. {@link StructuredCoder#registerByteSizeObserver} - * invokes {@link #getEncodedElementByteSize} which requires re-encoding an element - * unless it is overridden. This is considered expensive. - */ - @Override - public boolean isRegisterByteSizeObserverCheap(T value) { - return isRegisterByteSizeObserverCheap(value, Context.NESTED); - } - - /** - * {@inheritDoc} - * - * @return {@code false} unless it is overridden. {@link StructuredCoder#registerByteSizeObserver} - * invokes {@link #getEncodedElementByteSize} which requires re-encoding an element - * unless it is overridden. This is considered expensive. - */ - @Override - public boolean isRegisterByteSizeObserverCheap(T value, Context context) { - return false; - } - - /** - * Returns the size in bytes of the encoded value using this coder. - */ - protected long getEncodedElementByteSize(T value, Context context) - throws Exception { - try (CountingOutputStream os = new CountingOutputStream(ByteStreams.nullOutputStream())) { - encode(value, os, context); - return os.getCount(); - } catch (Exception exn) { - throw new IllegalArgumentException( - "Unable to encode element '" + value + "' with coder '" + this + "'.", exn); - } - } - - @Override - public void registerByteSizeObserver(T value, ElementByteSizeObserver observer) - throws Exception { - registerByteSizeObserver(value, observer, Context.NESTED); - } - - /** - * {@inheritDoc} - * - *

      For {@link StructuredCoder} subclasses, this notifies {@code observer} about the byte size - * of the encoded value using this coder as returned by {@link #getEncodedElementByteSize}. - */ - @Override - public void registerByteSizeObserver( - T value, ElementByteSizeObserver observer, Context context) - throws Exception { - observer.update(getEncodedElementByteSize(value, context)); - } - protected void verifyDeterministic(String message, Iterable> coders) throws NonDeterministicException { for (Coder coder : coders) { From 987c2cbca433f3e0ff12a637f2b0474e772c6beb Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 5 May 2017 10:13:05 -0700 Subject: [PATCH 081/387] Reparent many Coders to Atomic or StructuredCoder These coders do not take configuration, or take configuration only in terms of other Coders, and are appropriate to reparent. --- .../translation/utils/ApexStreamTuple.java | 5 ++- .../UnboundedReadFromBoundedSource.java | 4 +- .../runners/core/construction/CodersTest.java | 14 +------ .../core/construction/PCollectionsTest.java | 3 +- .../core/ElementAndRestrictionCoder.java | 17 +++++++- .../beam/runners/core/KeyedWorkItemCoder.java | 4 +- .../beam/runners/core/TimerInternals.java | 6 +-- .../direct/CloningBundleFactoryTest.java | 10 ++--- .../beam/runners/direct/DirectRunnerTest.java | 5 +-- .../UnboundedReadEvaluatorFactoryTest.java | 4 +- .../SingletonKeyedWorkItemCoder.java | 4 +- .../runners/dataflow/BatchViewOverrides.java | 8 ++-- .../runners/dataflow/internal/IsmFormat.java | 40 +++++++++++++++---- .../dataflow/util/RandomAccessData.java | 4 +- .../org/apache/beam/sdk/coders/AvroCoder.java | 19 +++++++++ .../beam/sdk/coders/BigDecimalCoder.java | 2 +- .../sdk/coders/BigEndianIntegerCoder.java | 2 +- .../beam/sdk/coders/BigEndianLongCoder.java | 4 +- .../beam/sdk/coders/BigIntegerCoder.java | 4 +- .../apache/beam/sdk/coders/BitSetCoder.java | 4 +- .../beam/sdk/coders/ByteArrayCoder.java | 6 +-- .../org/apache/beam/sdk/coders/ByteCoder.java | 2 +- .../beam/sdk/coders/CoderFactories.java | 9 +++-- .../apache/beam/sdk/coders/DoubleCoder.java | 2 +- .../apache/beam/sdk/coders/DurationCoder.java | 2 +- .../apache/beam/sdk/coders/InstantCoder.java | 2 +- .../org/apache/beam/sdk/coders/KvCoder.java | 4 +- .../org/apache/beam/sdk/coders/ListCoder.java | 3 +- .../org/apache/beam/sdk/coders/MapCoder.java | 2 +- .../apache/beam/sdk/coders/NullableCoder.java | 6 +-- .../beam/sdk/coders/StringUtf8Coder.java | 2 +- .../beam/sdk/coders/TextualIntegerCoder.java | 2 +- .../apache/beam/sdk/coders/VarIntCoder.java | 2 +- .../org/apache/beam/sdk/coders/VoidCoder.java | 4 +- .../org/apache/beam/sdk/io/FileBasedSink.java | 4 +- .../sdk/transforms/ApproximateQuantiles.java | 26 +++++++++--- .../apache/beam/sdk/transforms/Combine.java | 12 ++++-- .../beam/sdk/transforms/CombineFns.java | 4 +- .../org/apache/beam/sdk/transforms/Count.java | 4 +- .../org/apache/beam/sdk/transforms/Mean.java | 7 ++-- .../org/apache/beam/sdk/transforms/Top.java | 23 ++++++++++- .../beam/sdk/transforms/join/CoGbkResult.java | 2 +- .../beam/sdk/transforms/join/UnionCoder.java | 7 ++-- .../sdk/transforms/windowing/PaneInfo.java | 10 ++++- .../org/apache/beam/sdk/util/BitSetCoder.java | 9 +++-- .../apache/beam/sdk/util/WindowedValue.java | 7 ++-- .../beam/sdk/values/TimestampedValue.java | 13 ++++-- .../beam/sdk/values/ValueInSingleWindow.java | 4 +- .../beam/sdk/values/ValueWithRecordId.java | 4 +- .../beam/sdk/coders/CoderRegistryTest.java | 4 +- .../beam/sdk/coders/DelegateCoderTest.java | 25 ------------ .../beam/sdk/coders/NullableCoderTest.java | 2 +- .../beam/sdk/testing/CoderPropertiesTest.java | 37 +++++++++++++++-- .../apache/beam/sdk/testing/PAssertTest.java | 4 +- .../sdk/testing/SerializableMatchersTest.java | 4 +- .../beam/sdk/testing/WindowSupplierTest.java | 6 +-- .../beam/sdk/transforms/CombineFnsTest.java | 4 +- .../beam/sdk/transforms/CombineTest.java | 6 +-- .../beam/sdk/transforms/CreateTest.java | 6 +-- .../beam/sdk/transforms/GroupByKeyTest.java | 4 +- .../apache/beam/sdk/transforms/ParDoTest.java | 6 +-- .../apache/beam/sdk/transforms/ViewTest.java | 4 +- .../transforms/reflect/DoFnInvokersTest.java | 6 +-- .../apache/beam/sdk/util/CoderUtilsTest.java | 4 +- .../beam/sdk/util/SerializableUtilsTest.java | 4 +- .../extensions/protobuf/ByteStringCoder.java | 4 +- .../sdk/io/gcp/bigquery/ShardedKeyCoder.java | 10 ++--- .../gcp/bigquery/TableDestinationCoder.java | 4 +- .../io/gcp/bigquery/TableRowInfoCoder.java | 4 +- .../io/gcp/bigquery/TableRowJsonCoder.java | 4 +- .../io/gcp/bigquery/WriteBundlesToFiles.java | 11 ++++- .../io/gcp/pubsub/PubsubUnboundedSink.java | 4 +- .../io/gcp/pubsub/PubsubUnboundedSource.java | 12 ++++-- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 4 +- .../beam/sdk/io/hadoop/WritableCoder.java | 18 +++++++++ .../beam/sdk/io/hbase/HBaseMutationCoder.java | 4 +- .../beam/sdk/io/hbase/HBaseResultCoder.java | 4 +- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 6 +-- .../beam/sdk/io/kafka/KafkaRecordCoder.java | 4 +- .../sdk/io/kinesis/KinesisRecordCoder.java | 4 +- .../org/apache/beam/sdk/io/xml/JAXBCoder.java | 18 +++++++++ .../apache/beam/sdk/io/xml/JAXBCoderTest.java | 4 +- 82 files changed, 369 insertions(+), 223 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStreamTuple.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStreamTuple.java index 4ce351b882f3..4aa6ee82017e 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStreamTuple.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ApexStreamTuple.java @@ -31,7 +31,7 @@ import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.StructuredCoder; /** * The common interface for all objects transmitted through streams. @@ -149,7 +149,7 @@ public String toString() { /** * Coder for {@link ApexStreamTuple}. */ - class ApexStreamTupleCoder extends CustomCoder> { + class ApexStreamTupleCoder extends StructuredCoder> { private static final long serialVersionUID = 1L; final Coder valueCoder; @@ -194,6 +194,7 @@ public List> getCoderArguments() { @Override public void verifyDeterministic() throws NonDeterministicException { verifyDeterministic( + this, this.getClass().getSimpleName() + " requires a deterministic valueCoder", valueCoder); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java index 0ea13b8235d4..1424b8be2518 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java @@ -35,10 +35,10 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.Read; @@ -203,7 +203,7 @@ public void finalizeCheckpoint() {} } @VisibleForTesting - static class CheckpointCoder extends CustomCoder> { + static class CheckpointCoder extends StructuredCoder> { // The coder for a list of residual elements and their timestamps private final Coder>> elemsCoder; 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/CodersTest.java index 32a78faec70e..765723c9bb3d 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/CodersTest.java @@ -30,11 +30,11 @@ import java.io.Serializable; import java.util.HashSet; import java.util.Set; +import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.AvroCoder; 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.CustomCoder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.LengthPrefixCoder; @@ -149,7 +149,7 @@ public void toAndFromProto() throws Exception { static class Record implements Serializable {} - private static class RecordCoder extends CustomCoder { + private static class RecordCoder extends AtomicCoder { @Override public void encode(Record value, OutputStream outStream, Context context) throws CoderException, IOException {} @@ -159,16 +159,6 @@ public Record decode(InputStream inStream, Context context) throws CoderException, IOException { return new Record(); } - - @Override - public boolean equals(Object other) { - return other != null && getClass().equals(other.getClass()); - } - - @Override - public int hashCode() { - return getClass().hashCode(); - } } } } 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 c177c58cb6d6..2c45cbdb94b3 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 @@ -29,6 +29,7 @@ import java.util.Collection; import java.util.Collections; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CustomCoder; @@ -158,7 +159,7 @@ public boolean isCompatible(WindowFn other) { @Override public Coder windowCoder() { - return new CustomCoder() { + return new AtomicCoder() { @Override public void verifyDeterministic() {} @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java index 64c1e14156c0..83c4e6281d71 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementAndRestrictionCoder.java @@ -17,18 +17,20 @@ */ package org.apache.beam.runners.core; +import com.google.common.collect.ImmutableList; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.util.List; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.StructuredCoder; /** A {@link Coder} for {@link ElementAndRestriction}. */ @Experimental(Experimental.Kind.SPLITTABLE_DO_FN) public class ElementAndRestrictionCoder - extends CustomCoder> { + extends StructuredCoder> { private final Coder elementCoder; private final Coder restrictionCoder; @@ -65,6 +67,17 @@ public ElementAndRestriction decode(InputStream inStream return ElementAndRestriction.of(key, value); } + @Override + public List> getCoderArguments() { + return ImmutableList.of(elementCoder, restrictionCoder); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + elementCoder.verifyDeterministic(); + restrictionCoder.verifyDeterministic(); + } + public Coder getElementCoder() { return elementCoder; } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java index fddf7fa478e1..e1872b542260 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java @@ -26,8 +26,8 @@ import org.apache.beam.runners.core.TimerInternals.TimerDataCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; @@ -35,7 +35,7 @@ /** * A {@link Coder} for {@link KeyedWorkItem KeyedWorkItems}. */ -public class KeyedWorkItemCoder extends CustomCoder> { +public class KeyedWorkItemCoder extends StructuredCoder> { /** * Create a new {@link KeyedWorkItemCoder} with the provided key coder, element coder, and window * coder. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java index 21fe4301ea57..888c11f97313 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java @@ -27,9 +27,9 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -224,7 +224,7 @@ public int compareTo(TimerData that) { /** * A {@link Coder} for {@link TimerData}. */ - class TimerDataCoder extends CustomCoder { + class TimerDataCoder extends StructuredCoder { private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of(); private static final InstantCoder INSTANT_CODER = InstantCoder.of(); private final Coder windowCoder; @@ -266,7 +266,7 @@ public List> getCoderArguments() { @Override public void verifyDeterministic() throws NonDeterministicException { - verifyDeterministic("window coder must be deterministic", windowCoder); + verifyDeterministic(this, "window coder must be deterministic", windowCoder); } } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java index 7d037d124fd7..33d171e50449 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java @@ -31,8 +31,8 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; @@ -173,7 +173,7 @@ public void keyedBundleDecodeFailsAddFails() { } static class Record {} - static class RecordNoEncodeCoder extends CustomCoder { + static class RecordNoEncodeCoder extends AtomicCoder { @Override public void encode( @@ -192,7 +192,7 @@ public Record decode( } } - static class RecordNoDecodeCoder extends CustomCoder { + static class RecordNoDecodeCoder extends AtomicCoder { @Override public void encode( Record value, @@ -208,7 +208,7 @@ public Record decode( } } - private static class RecordStructuralValueCoder extends CustomCoder { + private static class RecordStructuralValueCoder extends AtomicCoder { @Override public void encode( Record value, @@ -240,7 +240,7 @@ public Object structuralValue(Record value) { } private static class RecordNotConsistentWithEqualsStructuralValueCoder - extends CustomCoder { + extends AtomicCoder { @Override public void encode( Record value, diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java index 428c6fc43d71..0fe958515465 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java @@ -44,9 +44,9 @@ import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.PipelineResult.State; import org.apache.beam.sdk.PipelineRunner; +import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; @@ -523,8 +523,7 @@ public void testUnencodableOutputFromUnboundedRead() { p.run(); } - private static class LongNoDecodeCoder extends CustomCoder { - + private static class LongNoDecodeCoder extends AtomicCoder { @Override public void encode( Long value, OutputStream outStream, Context context) throws IOException { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index ceb078be3058..b9ba7f49df79 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -46,10 +46,10 @@ import javax.annotation.Nullable; import org.apache.beam.runners.direct.UnboundedReadDeduplicator.NeverDeduplicator; import org.apache.beam.runners.direct.UnboundedReadEvaluatorFactory.UnboundedSourceShard; +import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.Read; @@ -586,7 +586,7 @@ boolean isFinalized() { return finalized; } - public static class Coder extends CustomCoder { + public static class Coder extends AtomicCoder { @Override public void encode( TestCheckpointMark value, diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java index c73700f9dc23..f21869341a01 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java @@ -26,7 +26,7 @@ import org.apache.beam.runners.core.KeyedWorkItemCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; @@ -34,7 +34,7 @@ * Singleton keyed work item coder. */ public class SingletonKeyedWorkItemCoder - extends CustomCoder> { + extends StructuredCoder> { /** * Create a new {@link KeyedWorkItemCoder} with the provided key coder, element coder, and window * coder. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index ef2bfed39e7c..ecd0365ed88c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -50,11 +50,11 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.MapCoder; import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -1335,7 +1335,7 @@ protected Map delegate() { * A {@link Coder} for {@link TransformedMap}s. */ static class TransformedMapCoder - extends CustomCoder> { + extends StructuredCoder> { private final Coder> transformCoder; private final Coder> originalMapCoder; @@ -1373,8 +1373,8 @@ public List> getCoderArguments() { @Override public void verifyDeterministic() throws org.apache.beam.sdk.coders.Coder.NonDeterministicException { - verifyDeterministic("Expected transform coder to be deterministic.", transformCoder); - verifyDeterministic("Expected map coder to be deterministic.", originalMapCoder); + verifyDeterministic(this, "Expected transform coder to be deterministic.", transformCoder); + verifyDeterministic(this, "Expected map coder to be deterministic.", originalMapCoder); } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java index fbfe49aaa5ad..aed514a08c7c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java @@ -32,14 +32,17 @@ import java.io.OutputStream; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.util.RandomAccessData; +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.Coder.NonDeterministicException; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.util.VarInt; @@ -356,8 +359,9 @@ public List> getCoderArguments() { @Override public void verifyDeterministic() throws Coder.NonDeterministicException { - verifyDeterministic("Key component coders expected to be deterministic.", keyComponentCoders); - verifyDeterministic("Value coder expected to be deterministic.", valueCoder); + verifyDeterministic( + this, "Key component coders expected to be deterministic.", keyComponentCoders); + verifyDeterministic(this, "Value coder expected to be deterministic.", valueCoder); } @Override @@ -393,6 +397,28 @@ public Object structuralValue(IsmRecord record) { } return super.structuralValue(record); } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + if (!(other instanceof IsmRecordCoder)) { + return false; + } + IsmRecordCoder that = (IsmRecordCoder) other; + return Objects.equals(this.numberOfShardKeyCoders, that.numberOfShardKeyCoders) + && Objects.equals( + this.numberOfMetadataShardKeyCoders, that.numberOfMetadataShardKeyCoders) + && Objects.equals(this.keyComponentCoders, that.keyComponentCoders) + && Objects.equals(this.valueCoder, that.valueCoder); + } + + @Override + public int hashCode() { + return Objects.hash( + numberOfShardKeyCoders, numberOfMetadataShardKeyCoders, keyComponentCoders, valueCoder); + } } /** @@ -450,7 +476,7 @@ public static Object getMetadataKey() { * A coder for metadata key component. Can be used to wrap key component coder allowing for * the metadata key component to be used as a place holder instead of an actual key. */ - public static class MetadataKeyCoder extends CustomCoder { + public static class MetadataKeyCoder extends StructuredCoder { public static MetadataKeyCoder of(Coder keyCoder) { checkNotNull(keyCoder); return new MetadataKeyCoder<>(keyCoder); @@ -497,7 +523,7 @@ public List> getCoderArguments() { @Override public void verifyDeterministic() throws NonDeterministicException { - verifyDeterministic("Expected key coder to be deterministic", keyCoder); + verifyDeterministic(this, "Expected key coder to be deterministic", keyCoder); } } @@ -584,7 +610,7 @@ public IsmShard withIndexOffset(long indexOffset) { *

    • indexOffset (variable length long encoding)
    • *
    */ - public static class IsmShardCoder extends CustomCoder { + public static class IsmShardCoder extends AtomicCoder { private static final IsmShardCoder INSTANCE = new IsmShardCoder(); /** Returns an IsmShardCoder. */ @@ -649,7 +675,7 @@ public static KeyPrefix of(int sharedKeySize, int unsharedKeySize) { } /** A {@link Coder} for {@link KeyPrefix}. */ - public static final class KeyPrefixCoder extends CustomCoder { + public static final class KeyPrefixCoder extends AtomicCoder { private static final KeyPrefixCoder INSTANCE = new KeyPrefixCoder(); public static KeyPrefixCoder of() { @@ -721,7 +747,7 @@ public static Footer of(long indexPosition, long bloomFilterPosition, long numbe } /** A {@link Coder} for {@link Footer}. */ - public static final class FooterCoder extends CustomCoder